You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2016/10/22 04:42:37 UTC
[1/2] kylin git commit: KYLIN-1919 performance enhancement
Repository: kylin
Updated Branches:
refs/heads/KYLIN-1971 5156ccd47 -> b1a965097
KYLIN-1919 performance enhancement
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/3142c74c
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/3142c74c
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/3142c74c
Branch: refs/heads/KYLIN-1971
Commit: 3142c74cb22ea2b80ba36de462da69197020e1b3
Parents: 5156ccd
Author: shaofengshi <sh...@apache.org>
Authored: Fri Oct 21 18:30:44 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Fri Oct 21 18:32:52 2016 +0800
----------------------------------------------------------------------
.../kylin/source/kafka/StreamingParser.java | 49 +++--
.../source/kafka/TimedJsonStreamParser.java | 31 ++--
.../test/java/TimedJsonStreamParserTest.java | 178 -------------------
.../source/kafka/TimedJsonStreamParserTest.java | 166 +++++++++++++++++
source-kafka/src/test/resources/message.json | 10 +-
5 files changed, 217 insertions(+), 217 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/3142c74c/source-kafka/src/main/java/org/apache/kylin/source/kafka/StreamingParser.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/StreamingParser.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/StreamingParser.java
index 43b2ac5..75f9c4b 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/StreamingParser.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/StreamingParser.java
@@ -21,7 +21,6 @@ package org.apache.kylin.source.kafka;
import java.lang.reflect.Constructor;
import java.util.List;
import java.util.Map;
-import java.util.Set;
import com.google.common.collect.Maps;
import org.apache.commons.lang3.StringUtils;
@@ -31,7 +30,6 @@ import org.apache.kylin.common.util.StreamingMessage;
import org.apache.kylin.common.util.TimeUtil;
import org.apache.kylin.metadata.model.TblColRef;
-import com.google.common.collect.Sets;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -47,15 +45,15 @@ public abstract class StreamingParser {
public static final String EMBEDDED_PROPERTY_SEPARATOR = "separator";
public static final Map<String, String> defaultProperties = Maps.newHashMap();
- public static final Set derivedTimeColumns = Sets.newHashSet();
+ public static final Map<String, Integer> derivedTimeColumns = Maps.newHashMap();
static {
- derivedTimeColumns.add("minute_start");
- derivedTimeColumns.add("hour_start");
- derivedTimeColumns.add("day_start");
- derivedTimeColumns.add("week_start");
- derivedTimeColumns.add("month_start");
- derivedTimeColumns.add("quarter_start");
- derivedTimeColumns.add("year_start");
+ derivedTimeColumns.put("minute_start", 1);
+ derivedTimeColumns.put("hour_start", 2);
+ derivedTimeColumns.put("day_start", 3);
+ derivedTimeColumns.put("week_start", 4);
+ derivedTimeColumns.put("month_start", 5);
+ derivedTimeColumns.put("quarter_start", 6);
+ derivedTimeColumns.put("year_start", 7);
defaultProperties.put(PROPERTY_TS_COLUMN_NAME, "timestamp");
defaultProperties.put(PROPERTY_TS_PARSER, "org.apache.kylin.source.kafka.DefaultTimeParser");
defaultProperties.put(PROPERTY_TS_PATTERN, DateFormat.DEFAULT_DATETIME_PATTERN_WITHOUT_MILLISECONDS);
@@ -108,32 +106,45 @@ public abstract class StreamingParser {
* @return true if the columnName is a derived time column; otherwise false;
*/
public static final boolean populateDerivedTimeColumns(String columnName, List<String> result, long t) {
- if (derivedTimeColumns.contains(columnName) == false)
+
+ Integer derivedTimeColumn = derivedTimeColumns.get(columnName);
+ if (derivedTimeColumn == null) {
return false;
+ }
long normalized = 0;
- if (columnName.equals("minute_start")) {
+ switch (derivedTimeColumn) {
+ case 1:
normalized = TimeUtil.getMinuteStart(t);
result.add(DateFormat.formatToTimeWithoutMilliStr(normalized));
- } else if (columnName.equals("hour_start")) {
+ break;
+ case 2:
normalized = TimeUtil.getHourStart(t);
result.add(DateFormat.formatToTimeWithoutMilliStr(normalized));
- } else if (columnName.equals("day_start")) {
- //from day_start on, formatTs will output date format
+ break;
+ case 3:
normalized = TimeUtil.getDayStart(t);
result.add(DateFormat.formatToDateStr(normalized));
- } else if (columnName.equals("week_start")) {
+ break;
+ case 4:
normalized = TimeUtil.getWeekStart(t);
result.add(DateFormat.formatToDateStr(normalized));
- } else if (columnName.equals("month_start")) {
+ break;
+ case 5:
normalized = TimeUtil.getMonthStart(t);
result.add(DateFormat.formatToDateStr(normalized));
- } else if (columnName.equals("quarter_start")) {
+ break;
+ case 6:
normalized = TimeUtil.getQuarterStart(t);
result.add(DateFormat.formatToDateStr(normalized));
- } else if (columnName.equals("year_start")) {
+ break;
+ case 7:
normalized = TimeUtil.getYearStart(t);
result.add(DateFormat.formatToDateStr(normalized));
+ break;
+ default:
+ throw new IllegalStateException();
+
}
return true;
http://git-wip-us.apache.org/repos/asf/kylin/blob/3142c74c/source-kafka/src/main/java/org/apache/kylin/source/kafka/TimedJsonStreamParser.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/main/java/org/apache/kylin/source/kafka/TimedJsonStreamParser.java b/source-kafka/src/main/java/org/apache/kylin/source/kafka/TimedJsonStreamParser.java
index 633a30c..e00ce16 100644
--- a/source-kafka/src/main/java/org/apache/kylin/source/kafka/TimedJsonStreamParser.java
+++ b/source-kafka/src/main/java/org/apache/kylin/source/kafka/TimedJsonStreamParser.java
@@ -64,6 +64,9 @@ public final class TimedJsonStreamParser extends StreamingParser {
private String tsColName = null;
private String tsParser = null;
private String separator = null;
+ private final Map<String, Object> root = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+ private final Map<String, Object> tempMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+ private final Map<String, String[]> nameMap = new HashMap<>();
private final JavaType mapType = MapType.construct(HashMap.class, SimpleType.construct(String.class), SimpleType.construct(Object.class));
@@ -100,15 +103,14 @@ public final class TimedJsonStreamParser extends StreamingParser {
public StreamingMessage parse(ByteBuffer buffer) {
try {
Map<String, Object> message = mapper.readValue(new ByteBufferBackedInputStream(buffer), mapType);
- Map<String, Object> root = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+ root.clear();
root.putAll(message);
String tsStr = objToString(root.get(tsColName));
long t = streamTimeParser.parseTime(tsStr);
ArrayList<String> result = Lists.newArrayList();
for (TblColRef column : allColumns) {
- String columnName = column.getName();
- columnName = columnName.toLowerCase();
+ final String columnName = column.getName().toLowerCase();
if (populateDerivedTimeColumns(columnName, result, t) == false) {
result.add(getValueByKey(columnName, root));
}
@@ -126,18 +128,24 @@ public final class TimedJsonStreamParser extends StreamingParser {
return true;
}
- protected String getValueByKey(String key, Map<String, Object> root) throws IOException {
- if (root.containsKey(key)) {
- return objToString(root.get(key));
+ protected String getValueByKey(String key, Map<String, Object> rootMap) throws IOException {
+ if (rootMap.containsKey(key)) {
+ return objToString(rootMap.get(key));
}
- if (key.contains(separator)) {
- String[] names = key.toLowerCase().split(separator);
- Map<String, Object> tempMap = root;
+ String[] names = nameMap.get(key);
+ if (names == null && key.contains(separator)) {
+ names = key.toLowerCase().split(separator);
+ nameMap.put(key, names);
+ }
+
+ if (names != null && names.length > 0) {
+ tempMap.clear();
+ tempMap.putAll(rootMap);
for (int i = 0; i < names.length - 1; i++) {
- Object o = root.get(names[i]);
+ Object o = tempMap.get(names[i]);
if (o instanceof Map) {
- tempMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
+ tempMap.clear();
tempMap.putAll((Map<String, Object>) o);
} else {
throw new IOException("Property '" + names[i] + "' is not embedded format");
@@ -145,7 +153,6 @@ public final class TimedJsonStreamParser extends StreamingParser {
}
Object finalObject = tempMap.get(names[names.length - 1]);
return objToString(finalObject);
-
}
return StringUtils.EMPTY;
http://git-wip-us.apache.org/repos/asf/kylin/blob/3142c74c/source-kafka/src/test/java/TimedJsonStreamParserTest.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/test/java/TimedJsonStreamParserTest.java b/source-kafka/src/test/java/TimedJsonStreamParserTest.java
deleted file mode 100644
index 5a52b61..0000000
--- a/source-kafka/src/test/java/TimedJsonStreamParserTest.java
+++ /dev/null
@@ -1,178 +0,0 @@
-/*
- * 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.
- */
-
-import com.fasterxml.jackson.databind.JavaType;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import com.fasterxml.jackson.databind.type.MapType;
-import com.fasterxml.jackson.databind.type.SimpleType;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.kylin.common.util.StreamingMessage;
-import org.apache.kylin.metadata.model.ColumnDesc;
-import org.apache.kylin.metadata.model.TableDesc;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.source.kafka.TimedJsonStreamParser;
-
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.List;
-import java.util.HashMap;
-import java.util.ArrayList;
-
-
-import static org.junit.Assert.assertEquals;
-
-import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-
-
-public class TimedJsonStreamParserTest extends LocalFileMetadataTestCase {
-
- private static String[] userNeedColNames;
-
- private static final String jsonFilePath = "src/test/resources/message.json";
-
- private static ObjectMapper mapper;
-
- private final JavaType mapType = MapType.construct(HashMap.class, SimpleType.construct(String.class),
- SimpleType.construct(Object.class));
-
-
- @BeforeClass
- public static void setUp() throws Exception {
- staticCreateTestMetadata();
- mapper = new ObjectMapper();
- }
-
- @AfterClass
- public static void after() throws Exception {
- cleanAfterClass();
- }
-
-
- @Test
- public void testNormalValue() throws Exception {
- userNeedColNames = new String[]{"createdAt", "id", "isTruncated", "text"};
- List<TblColRef> allCol = mockupTblColRefList();
- TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
- Object msg = mapper.readValue(new File(jsonFilePath), mapType);
- ByteBuffer buffer = getJsonByteBuffer(msg);
- StreamingMessage sMsg = parser.parse(buffer);
- List<String> result = sMsg.getData();
- assertEquals("Jul 20, 2016 9:59:17 AM", result.get(0));
- assertEquals("755703618762862600", result.get(1));
- assertEquals("false", result.get(2));
- assertEquals("dejamos las tapas regionales de este #Miercoles https://t.co/kfe0kT2Fup", result.get(3));
- }
-
- @Test
- public void testEmbeddedValue() throws Exception {
- userNeedColNames = new String[]{"user_id", "user_description", "user_isProtected"};
- List<TblColRef> allCol = mockupTblColRefList();
- TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
- Object msg = mapper.readValue(new File(jsonFilePath), mapType);
- ByteBuffer buffer = getJsonByteBuffer(msg);
- StreamingMessage sMsg = parser.parse(buffer);
- List<String> result = sMsg.getData();
- assertEquals("4853763947", result.get(0));
- assertEquals("Noticias, an\ufffd\ufffdlisis e informaci\ufffd\ufffdn para el crecimiento de la regi\ufffd\ufffdn.", result.get(1));
- assertEquals("false", result.get(2));
- }
-
- @Test
- public void testArrayValue() throws Exception {
- userNeedColNames = new String[]{"userMentionEntities", "mediaEntities"};
- List<TblColRef> allCol = mockupTblColRefList();
- TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
- Object msg = mapper.readValue(new File(jsonFilePath), mapType);
- HashMap<String, Object> map = (HashMap<String, Object>) msg;
- Object array = map.get("mediaEntities");
- ByteBuffer buffer = getJsonByteBuffer(msg);
- StreamingMessage sMsg = parser.parse(buffer);
- List<String> result = sMsg.getData();
- System.out.println(result);
-
- }
-
- @Test
- public void testMapValue() throws Exception {
- userNeedColNames = new String[]{"user"};
- List<TblColRef> allCol = mockupTblColRefList();
- TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
- Object msg = mapper.readValue(new File(jsonFilePath), mapType);
- ByteBuffer buffer = getJsonByteBuffer(msg);
- StreamingMessage sMsg = parser.parse(buffer);
- List<String> result = sMsg.getData();
- System.out.println("result:" + result);
-
- }
-
- @Test
- public void testNullKey() throws Exception {
- userNeedColNames = new String[]{"null", ""};
- List<TblColRef> allCol = mockupTblColRefList();
- TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
- Object msg = mapper.readValue(new File(jsonFilePath), mapType);
- ByteBuffer buffer = getJsonByteBuffer(msg);
- StreamingMessage sMsg = parser.parse(buffer);
- List<String> result = sMsg.getData();
- assertEquals(StringUtils.EMPTY, result.get(0));
- assertEquals(StringUtils.EMPTY, result.get(1));
- }
-
-
- private static ByteBuffer getJsonByteBuffer(Object obj) throws IOException {
- byte[] bytes = mapper.writeValueAsBytes(obj);
- ByteBuffer buff = ByteBuffer.wrap(bytes);
- buff.position(0);
- return buff;
- }
-
-
- private static List<TblColRef> mockupTblColRefList() {
- TableDesc t = mockupTableDesc("table_a");
- List<TblColRef> list = new ArrayList<>();
- for (int i = 0; i < userNeedColNames.length; i++) {
- ColumnDesc c = mockupColumnDesc(t, i, userNeedColNames[i], "string");
- list.add(c.getRef());
- }
- return list;
- }
-
- private static TableDesc mockupTableDesc(String tableName) {
- TableDesc mockup = new TableDesc();
- mockup.setName(tableName);
- return mockup;
- }
-
- private static ColumnDesc mockupColumnDesc(TableDesc table, int oneBasedColumnIndex, String name, String datatype) {
- ColumnDesc desc = new ColumnDesc();
- String id = "" + oneBasedColumnIndex;
- desc.setId(id);
- desc.setName(name);
- desc.setDatatype(datatype);
- desc.init(table);
- return desc;
- }
-}
http://git-wip-us.apache.org/repos/asf/kylin/blob/3142c74c/source-kafka/src/test/java/org/apache/kylin/source/kafka/TimedJsonStreamParserTest.java
----------------------------------------------------------------------
diff --git a/source-kafka/src/test/java/org/apache/kylin/source/kafka/TimedJsonStreamParserTest.java b/source-kafka/src/test/java/org/apache/kylin/source/kafka/TimedJsonStreamParserTest.java
new file mode 100644
index 0000000..f92a24e
--- /dev/null
+++ b/source-kafka/src/test/java/org/apache/kylin/source/kafka/TimedJsonStreamParserTest.java
@@ -0,0 +1,166 @@
+/*
+ * 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.source.kafka;
+
+import com.fasterxml.jackson.databind.JavaType;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import com.fasterxml.jackson.databind.type.MapType;
+import com.fasterxml.jackson.databind.type.SimpleType;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.kylin.common.util.StreamingMessage;
+import org.apache.kylin.metadata.model.ColumnDesc;
+import org.apache.kylin.metadata.model.TableDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.HashMap;
+import java.util.ArrayList;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.kylin.common.util.LocalFileMetadataTestCase;
+
+public class TimedJsonStreamParserTest extends LocalFileMetadataTestCase {
+
+ private static String[] userNeedColNames;
+ private static final String jsonFilePath = "src/test/resources/message.json";
+ private static ObjectMapper mapper;
+ private final JavaType mapType = MapType.construct(HashMap.class, SimpleType.construct(String.class), SimpleType.construct(Object.class));
+
+ @BeforeClass
+ public static void setUp() throws Exception {
+ staticCreateTestMetadata();
+ mapper = new ObjectMapper();
+ }
+
+ @AfterClass
+ public static void after() throws Exception {
+ cleanAfterClass();
+ }
+
+ @Test
+ public void testNormalValue() throws Exception {
+ userNeedColNames = new String[] { "createdAt", "id", "isTruncated", "text" };
+ List<TblColRef> allCol = mockupTblColRefList();
+ TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
+ Object msg = mapper.readValue(new File(jsonFilePath), mapType);
+ ByteBuffer buffer = getJsonByteBuffer(msg);
+ StreamingMessage sMsg = parser.parse(buffer);
+ List<String> result = sMsg.getData();
+ assertEquals("Jul 20, 2016 9:59:17 AM", result.get(0));
+ assertEquals("755703618762862600", result.get(1));
+ assertEquals("false", result.get(2));
+ assertEquals("dejamos", result.get(3));
+ }
+
+ @Test
+ public void testEmbeddedValue() throws Exception {
+ userNeedColNames = new String[] { "user_id", "user_description", "user_isProtected" };
+ List<TblColRef> allCol = mockupTblColRefList();
+ TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
+ Object msg = mapper.readValue(new File(jsonFilePath), mapType);
+ ByteBuffer buffer = getJsonByteBuffer(msg);
+ StreamingMessage sMsg = parser.parse(buffer);
+ List<String> result = sMsg.getData();
+ assertEquals("4853763947", result.get(0));
+ assertEquals("Noticias", result.get(1));
+ assertEquals("false", result.get(2));
+ }
+
+ @Test
+ public void testArrayValue() throws Exception {
+ userNeedColNames = new String[] { "userMentionEntities", "mediaEntities" };
+ List<TblColRef> allCol = mockupTblColRefList();
+ TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
+ Object msg = mapper.readValue(new File(jsonFilePath), mapType);
+ HashMap<String, Object> map = (HashMap<String, Object>) msg;
+ Object array = map.get("mediaEntities");
+ ByteBuffer buffer = getJsonByteBuffer(msg);
+ StreamingMessage sMsg = parser.parse(buffer);
+ List<String> result = sMsg.getData();
+ System.out.println(result);
+
+ }
+
+ @Test
+ public void testMapValue() throws Exception {
+ userNeedColNames = new String[] { "user" };
+ List<TblColRef> allCol = mockupTblColRefList();
+ TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
+ Object msg = mapper.readValue(new File(jsonFilePath), mapType);
+ ByteBuffer buffer = getJsonByteBuffer(msg);
+ StreamingMessage sMsg = parser.parse(buffer);
+ List<String> result = sMsg.getData();
+
+ }
+
+ @Test
+ public void testNullKey() throws Exception {
+ userNeedColNames = new String[] { "null", "" };
+ List<TblColRef> allCol = mockupTblColRefList();
+ TimedJsonStreamParser parser = new TimedJsonStreamParser(allCol, null);
+ Object msg = mapper.readValue(new File(jsonFilePath), mapType);
+ ByteBuffer buffer = getJsonByteBuffer(msg);
+ StreamingMessage sMsg = parser.parse(buffer);
+ List<String> result = sMsg.getData();
+ assertEquals(StringUtils.EMPTY, result.get(0));
+ assertEquals(StringUtils.EMPTY, result.get(1));
+ }
+
+ private static ByteBuffer getJsonByteBuffer(Object obj) throws IOException {
+ byte[] bytes = mapper.writeValueAsBytes(obj);
+ ByteBuffer buff = ByteBuffer.wrap(bytes);
+ buff.position(0);
+ return buff;
+ }
+
+ private static List<TblColRef> mockupTblColRefList() {
+ TableDesc t = mockupTableDesc("table_a");
+ List<TblColRef> list = new ArrayList<>();
+ for (int i = 0; i < userNeedColNames.length; i++) {
+ ColumnDesc c = mockupColumnDesc(t, i, userNeedColNames[i], "string");
+ list.add(c.getRef());
+ }
+ return list;
+ }
+
+ private static TableDesc mockupTableDesc(String tableName) {
+ TableDesc mockup = new TableDesc();
+ mockup.setName(tableName);
+ return mockup;
+ }
+
+ private static ColumnDesc mockupColumnDesc(TableDesc table, int oneBasedColumnIndex, String name, String datatype) {
+ ColumnDesc desc = new ColumnDesc();
+ String id = "" + oneBasedColumnIndex;
+ desc.setId(id);
+ desc.setName(name);
+ desc.setDatatype(datatype);
+ desc.init(table);
+ return desc;
+ }
+}
http://git-wip-us.apache.org/repos/asf/kylin/blob/3142c74c/source-kafka/src/test/resources/message.json
----------------------------------------------------------------------
diff --git a/source-kafka/src/test/resources/message.json b/source-kafka/src/test/resources/message.json
index dfafd45..55f35d3 100644
--- a/source-kafka/src/test/resources/message.json
+++ b/source-kafka/src/test/resources/message.json
@@ -1,8 +1,7 @@
{
"createdAt": "Jul 20, 2016 9:59:17 AM",
"id": 755703618762862600,
- "text": "dejamos las tapas regionales de este #Miercoles https://t.co/kfe0kT2Fup",
- "source": "<a href=\"http://twitter.com\" rel=\"nofollow\">Twitter Web Client</a>",
+ "text": "dejamos",
"isTruncated": false,
"inReplyToStatusId": -1,
"inReplyToUserId": -1,
@@ -15,8 +14,6 @@
"mediaEntities": [
{
"id": 755703584084328400,
- "url": "https://t.co/kfe0kT2Fup",
- "displayURL": "pic.twitter.com/kfe0kT2Fup",
"sizes": {
"0": {
"width": 150,
@@ -38,11 +35,8 @@
"currentUserRetweetId": -1,
"user": {
"id": 4853763947,
- "name": "El Metropolitano",
- "screenName": "ElTWdelMetro",
- "description": "Noticias, an\ufffd\ufffdlisis e informaci\ufffd\ufffdn para el crecimiento de la regi\ufffd\ufffdn.",
+ "description": "Noticias",
"isDefaultProfileImage": false,
- "url": "http://elmetropolitano.com.ar/",
"isProtected": false
}
}
\ No newline at end of file
[2/2] kylin git commit: Cube meta upgrade
Posted by li...@apache.org.
Cube meta upgrade
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/b1a96509
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/b1a96509
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/b1a96509
Branch: refs/heads/KYLIN-1971
Commit: b1a965097d739b783759100804cf7d9af7f665a0
Parents: 3142c74
Author: Yang Li <li...@apache.org>
Authored: Sat Oct 22 12:42:19 2016 +0800
Committer: Yang Li <li...@apache.org>
Committed: Sat Oct 22 12:42:19 2016 +0800
----------------------------------------------------------------------
.../org/apache/kylin/cube/CubeDescManager.java | 14 ++-
.../java/org/apache/kylin/cube/CubeManager.java | 18 ----
.../kylin/cube/model/AggregationGroup.java | 48 ++++++----
.../org/apache/kylin/cube/model/CubeDesc.java | 97 +++++---------------
.../apache/kylin/cube/model/DimensionDesc.java | 42 +++------
.../apache/kylin/cube/model/RowKeyColDesc.java | 10 +-
.../org/apache/kylin/cube/model/RowKeyDesc.java | 17 ++--
.../kylin/cube/model/v1_4_0/CubeDesc.java | 4 +-
.../org/apache/kylin/cube/CubeDescTest.java | 27 +++---
.../validation/rule/DictionaryRuleTest.java | 7 +-
.../model/validation/rule/FunctionRuleTest.java | 21 ++---
.../kylin/metadata/model/FunctionDesc.java | 21 +----
.../hbase/util/ExtendCubeToHybridCLI.java | 2 +-
13 files changed, 108 insertions(+), 220 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java
index d6364fe..50312bf 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeDescManager.java
@@ -32,7 +32,6 @@ import org.apache.kylin.cube.model.CubeDesc;
import org.apache.kylin.cube.model.validation.CubeMetadataValidator;
import org.apache.kylin.cube.model.validation.ValidateContext;
import org.apache.kylin.metadata.MetadataConstants;
-import org.apache.kylin.metadata.MetadataManager;
import org.apache.kylin.metadata.cachesync.Broadcaster;
import org.apache.kylin.metadata.cachesync.Broadcaster.Event;
import org.apache.kylin.metadata.cachesync.CaseInsensitiveStringCache;
@@ -176,8 +175,9 @@ public class CubeDescManager {
throw new IllegalArgumentException("No cube desc found at " + path);
try {
- ndesc.init(config, getMetadataManager().getAllTablesMap());
+ ndesc.init(config);
} catch (Exception e) {
+ logger.warn("Broken cube desc " + path, e);
ndesc.addError(e.getMessage());
}
@@ -202,8 +202,9 @@ public class CubeDescManager {
throw new IllegalArgumentException("CubeDesc '" + cubeDesc.getName() + "' already exists");
try {
- cubeDesc.init(config, getMetadataManager().getAllTablesMap());
+ cubeDesc.init(config);
} catch (Exception e) {
+ logger.warn("Broken cube desc " + cubeDesc, e);
cubeDesc.addError(e.getMessage());
}
// Check base validation
@@ -283,8 +284,9 @@ public class CubeDescManager {
}
try {
- desc.init(config, getMetadataManager().getAllTablesMap());
+ desc.init(config);
} catch (Exception e) {
+ logger.warn("Broken cube desc " + desc, e);
desc.addError(e.getMessage());
return desc;
}
@@ -310,10 +312,6 @@ public class CubeDescManager {
return ndesc;
}
- private MetadataManager getMetadataManager() {
- return MetadataManager.getInstance(config);
- }
-
private ResourceStore getStore() {
return ResourceStore.getStore(this.config);
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
index c558c6d..a53849e 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeManager.java
@@ -647,24 +647,6 @@ public class CubeManager implements IRealizationProvider {
}
}
- private long calculateStartOffsetForAppendSegment(CubeInstance cube) {
- List<CubeSegment> existing = cube.getSegments();
- if (existing.isEmpty()) {
- return 0;
- } else {
- return existing.get(existing.size() - 1).getSourceOffsetEnd();
- }
- }
-
- private long calculateStartDateForAppendSegment(CubeInstance cube) {
- List<CubeSegment> existing = cube.getSegments();
- if (existing.isEmpty()) {
- return cube.getDescriptor().getPartitionDateStart();
- } else {
- return existing.get(existing.size() - 1).getDateRangeEnd();
- }
- }
-
private void checkBuildingSegment(CubeInstance cube) {
int maxBuldingSeg = cube.getConfig().getMaxBuildingSegments();
if (cube.getBuildingSegments().size() >= maxBuldingSeg) {
http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java b/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java
index 601ee0a..6e76ac3 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java
@@ -20,8 +20,8 @@ package org.apache.kylin.cube.model;
import java.util.ArrayList;
import java.util.List;
-import java.util.Map;
+import org.apache.kylin.common.util.StringUtil;
import org.apache.kylin.cube.cuboid.Cuboid;
import org.apache.kylin.metadata.model.TblColRef;
@@ -59,36 +59,52 @@ public class AggregationGroup {
public void init(CubeDesc cubeDesc, RowKeyDesc rowKeyDesc) {
this.cubeDesc = cubeDesc;
this.isMandatoryOnlyValid = cubeDesc.getConfig().getCubeAggrGroupIsMandatoryOnlyValid();
- Map<String, TblColRef> colNameAbbr = cubeDesc.buildColumnNameAbbreviation();
if (this.includes == null || this.includes.length == 0 || this.selectRule == null) {
throw new IllegalStateException("AggregationGroup incomplete");
}
- buildPartialCubeFullMask(colNameAbbr, rowKeyDesc);
- buildMandatoryColumnMask(colNameAbbr, rowKeyDesc);
- buildHierarchyMasks(colNameAbbr, rowKeyDesc);
- buildJointColumnMask(colNameAbbr, rowKeyDesc);
+ columnNamesToUpperCase();
+
+ buildPartialCubeFullMask(rowKeyDesc);
+ buildMandatoryColumnMask(rowKeyDesc);
+ buildHierarchyMasks(rowKeyDesc);
+ buildJointColumnMask(rowKeyDesc);
buildJointDimsMask();
buildNormalDimsMask();
buildHierarchyDimsMask();
}
- private void buildPartialCubeFullMask(Map<String, TblColRef> colNameAbbr, RowKeyDesc rowKeyDesc) {
+ private void columnNamesToUpperCase() {
+ StringUtil.toUpperCaseArray(includes, includes);
+ StringUtil.toUpperCaseArray(selectRule.mandatory_dims, selectRule.mandatory_dims);
+ if (selectRule.hierarchy_dims != null) {
+ for (String[] cols : selectRule.hierarchy_dims) {
+ StringUtil.toUpperCaseArray(cols, cols);
+ }
+ }
+ if (selectRule.joint_dims != null) {
+ for (String[] cols : selectRule.joint_dims) {
+ StringUtil.toUpperCaseArray(cols, cols);
+ }
+ }
+ }
+
+ private void buildPartialCubeFullMask(RowKeyDesc rowKeyDesc) {
Preconditions.checkState(this.includes != null);
Preconditions.checkState(this.includes.length != 0);
partialCubeFullMask = 0L;
for (String dim : this.includes) {
- TblColRef hColumn = colNameAbbr.get(dim);
+ TblColRef hColumn = cubeDesc.getModel().findColumn(dim);
Integer index = rowKeyDesc.getColumnBitIndex(hColumn);
long bit = 1L << index;
partialCubeFullMask |= bit;
}
}
- private void buildJointColumnMask(Map<String, TblColRef> colNameAbbr, RowKeyDesc rowKeyDesc) {
+ private void buildJointColumnMask(RowKeyDesc rowKeyDesc) {
joints = Lists.newArrayList();
if (this.selectRule.joint_dims == null || this.selectRule.joint_dims.length == 0) {
@@ -102,7 +118,7 @@ public class AggregationGroup {
long joint = 0L;
for (int i = 0; i < joint_dims.length; i++) {
- TblColRef hColumn = colNameAbbr.get(joint_dims[i]);
+ TblColRef hColumn = cubeDesc.getModel().findColumn(joint_dims[i]);
Integer index = rowKeyDesc.getColumnBitIndex(hColumn);
long bit = 1L << index;
joint |= bit;
@@ -113,7 +129,7 @@ public class AggregationGroup {
}
}
- private void buildMandatoryColumnMask(Map<String, TblColRef> colNameAbbr, RowKeyDesc rowKeyDesc) {
+ private void buildMandatoryColumnMask(RowKeyDesc rowKeyDesc) {
mandatoryColumnMask = 0L;
String[] mandatory_dims = this.selectRule.mandatory_dims;
@@ -122,14 +138,14 @@ public class AggregationGroup {
}
for (String dim : mandatory_dims) {
- TblColRef hColumn = colNameAbbr.get(dim);
+ TblColRef hColumn = cubeDesc.getModel().findColumn(dim);
Integer index = rowKeyDesc.getColumnBitIndex(hColumn);
mandatoryColumnMask |= 1 << index;
}
}
- private void buildHierarchyMasks(Map<String, TblColRef> colNameAbbr, RowKeyDesc rowKeyDesc) {
+ private void buildHierarchyMasks(RowKeyDesc rowKeyDesc) {
this.hierarchyMasks = new ArrayList<HierarchyMask>();
if (this.selectRule.hierarchy_dims == null || this.selectRule.hierarchy_dims.length == 0) {
@@ -145,14 +161,10 @@ public class AggregationGroup {
ArrayList<Long> allMaskList = new ArrayList<Long>();
ArrayList<Long> dimList = new ArrayList<Long>();
for (int i = 0; i < hierarchy_dims.length; i++) {
- TblColRef hColumn = colNameAbbr.get(hierarchy_dims[i]);
+ TblColRef hColumn = cubeDesc.getModel().findColumn(hierarchy_dims[i]);
Integer index = rowKeyDesc.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);
dimList.add(bit);
http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
index 7db460e..64360d5 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeDesc.java
@@ -49,13 +49,11 @@ import org.apache.kylin.common.KylinVersion;
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.measure.MeasureType;
import org.apache.kylin.measure.extendedcolumn.ExtendedColumnMeasureType;
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;
@@ -63,6 +61,7 @@ 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.TableRef;
import org.apache.kylin.metadata.model.TblColRef;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -161,7 +160,6 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
@JsonProperty("override_kylin_properties")
private LinkedHashMap<String, String> overrideKylinProps = new LinkedHashMap<String, String>();
- 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>();
@@ -233,11 +231,7 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
}
public TblColRef findColumnRef(String table, String column) {
- Map<String, TblColRef> cols = columnMap.get(table);
- if (cols == null)
- return null;
- else
- return cols.get(column);
+ return model.findColumn(table, column);
}
public DimensionDesc findDimensionByTable(String lookupTableName) {
@@ -522,15 +516,7 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
}
}
- public Map<String, TblColRef> buildColumnNameAbbreviation() {
- Map<String, TblColRef> r = new CaseInsensitiveStringMap<TblColRef>();
- for (TblColRef col : listDimensionColumnsExcludingDerived(true)) {
- r.put(col.getName(), col);
- }
- return r;
- }
-
- public void init(KylinConfig config, Map<String, TableDesc> tables) {
+ public void init(KylinConfig config) {
this.errors.clear();
this.config = KylinConfigExt.createInstance(config, overrideKylinProps);
@@ -540,11 +526,8 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
this.model = MetadataManager.getInstance(config).getDataModelDesc(modelName);
checkNotNull(this.model, "DateModelDesc(%s) not found", modelName);
- // check if aggregation group is valid
- validate();
-
for (DimensionDesc dim : dimensions) {
- dim.init(this, tables);
+ dim.init(this);
}
initDimensionColumns();
@@ -554,6 +537,7 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
for (AggregationGroup agg : this.aggregationGroups) {
agg.init(this, rowkey);
}
+ validateAggregationGroups(); // check if aggregation group is valid
if (hbaseMapping != null) {
hbaseMapping.init(this);
@@ -563,14 +547,12 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
// check all dimension columns are presented on rowkey
List<TblColRef> dimCols = listDimensionColumnsExcludingDerived(true);
- checkState(rowkey.getRowKeyColumns().length == dimCols.size(),
- "RowKey columns count (%d) doesn't match dimensions columns count (%d)",
- rowkey.getRowKeyColumns().length, dimCols.size());
+ checkState(rowkey.getRowKeyColumns().length == dimCols.size(), "RowKey columns count (%d) doesn't match dimensions columns count (%d)", rowkey.getRowKeyColumns().length, dimCols.size());
initDictionaryDesc();
}
- public void validate() {
+ public void validateAggregationGroups() {
int index = 0;
for (AggregationGroup agg : getAggregationGroups()) {
@@ -706,27 +688,17 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
// init dimension columns
ArrayList<TblColRef> dimCols = Lists.newArrayList();
- String colStrs = dim.getColumn();
+ String colStr = dim.getColumn();
- if ((colStrs == null && dim.isDerived()) || ("{FK}".equalsIgnoreCase(colStrs))) {
+ if ((colStr == null && dim.isDerived()) || ("{FK}".equalsIgnoreCase(colStr))) {
// when column is omitted, special case
-
for (TblColRef col : join.getForeignKeyColumns()) {
dimCols.add(initDimensionColRef(col));
}
} else {
// normal case
-
- if (StringUtils.isEmpty(colStrs))
- throw new IllegalStateException("Dimension column must not be blank " + dim);
-
- dimCols.add(initDimensionColRef(dim, colStrs));
-
- // // fill back column ref in hierarchy
- // if (dim.isHierarchy()) {
- // for (int i = 0; i < dimCols.size(); i++)
- // dim.getHierarchy()[i].setColumnRef(dimCols.get(i));
- // }
+ checkState(!StringUtils.isEmpty(colStr), "Dimension column must not be blank: %s", dim);
+ dimCols.add(initDimensionColRef(dim, colStr));
}
TblColRef[] dimColArray = dimCols.toArray(new TblColRef[dimCols.size()]);
@@ -759,15 +731,6 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
initDerivedMap(new TblColRef[] { dimColArray[find] }, DeriveType.PK_FK, dim, new TblColRef[] { derivedCol }, null);
}
}
- /** 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);
- }
- }
- */
}
}
}
@@ -822,39 +785,25 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
}
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 = col.getRef();
+ TableRef table = dim.getTableRef();
+ TblColRef col = table.getColumn(colName);
+ checkArgument(col != null, "No column '%s' found in table %s", colName, table);
// 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);
+ int idx = ArrayUtils.indexOf(join.getPrimaryKeyColumns(), col);
if (idx >= 0) {
- ref = join.getForeignKeyColumns()[idx];
+ col = join.getForeignKeyColumns()[idx];
}
}
- return initDimensionColRef(ref);
+ return initDimensionColRef(col);
}
- 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 TblColRef initDimensionColRef(TblColRef col) {
+ allColumns.add(col);
+ dimensionColumns.add(col);
+ return col;
}
private void initMeasureColumns() {
@@ -862,8 +811,6 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
return;
}
- TableDesc factTable = getFactTableDesc();
- List<TableDesc> lookupTables = getLookupTableDescs();
for (MeasureDesc m : measures) {
m.setName(m.getName().toUpperCase());
@@ -872,7 +819,7 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
}
FunctionDesc func = m.getFunction();
- func.init(factTable, lookupTables);
+ func.init(model);
allColumns.addAll(func.getParameter().getColRefs());
if (ExtendedColumnMeasureType.FUNC_RAW.equalsIgnoreCase(m.getFunction().getExpression())) {
http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/main/java/org/apache/kylin/cube/model/DimensionDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/DimensionDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/DimensionDesc.java
index 0214ff0..ae90a18 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/DimensionDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/DimensionDesc.java
@@ -19,13 +19,13 @@
package org.apache.kylin.cube.model;
import java.util.Arrays;
-import java.util.Map;
import org.apache.commons.lang.NotImplementedException;
import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.metadata.model.DataModelDesc;
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.TableRef;
import org.apache.kylin.metadata.model.TblColRef;
import com.fasterxml.jackson.annotation.JsonAutoDetect;
@@ -47,52 +47,32 @@ public class DimensionDesc {
@JsonProperty("derived")
private String[] derived;
- private TableDesc tableDesc;
+ private TableRef tableRef;
private JoinDesc join;
// computed
private TblColRef[] columnRefs;
- public void init(CubeDesc cubeDesc, Map<String, TableDesc> tables) {
+ public void init(CubeDesc cubeDesc) {
if (name != null)
name = name.toUpperCase();
if (table != null)
table = table.toUpperCase();
- tableDesc = tables.get(this.getTable());
- if (tableDesc == null)
+ DataModelDesc model = cubeDesc.getModel();
+ tableRef = model.findTable(this.getTable());
+ if (tableRef == 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())) {
+ for (LookupDesc lookup : model.getLookups()) {
+ if (lookup.getTableRef().equals(this.tableRef)) {
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 (hierarchy != null) {
- // for (HierarchyDesc h : hierarchy)
- // h.setColumn(h.getColumn().toUpperCase());
- // }
-
if (derived != null && derived.length == 0) {
derived = null;
}
@@ -153,8 +133,8 @@ public class DimensionDesc {
this.derived = derived;
}
- public TableDesc getTableDesc() {
- return this.tableDesc;
+ public TableRef getTableRef() {
+ return this.tableRef;
}
@Override
http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/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 12c4dfc..9e2cb48 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,8 +18,6 @@
package org.apache.kylin.cube.model;
-import java.util.Map;
-
import org.apache.commons.lang.StringUtils;
import org.apache.kylin.dimension.DateDimEnc;
import org.apache.kylin.dimension.DictionaryDimEnc;
@@ -59,13 +57,11 @@ public class RowKeyColDesc {
private int bitIndex;
private TblColRef colRef;
- public void init(int index, Map<String, TblColRef> colNameAbbr, CubeDesc cubeDesc) {
+ public void init(int index, CubeDesc cubeDesc) {
column = column.toUpperCase();
bitIndex = index;
- colRef = colNameAbbr.get(column);
- if (colRef == null) {
- throw new IllegalArgumentException("Cannot find rowkey column " + column + " in cube " + cubeDesc);
- }
+ colRef = cubeDesc.getModel().findColumn(column);
+ Preconditions.checkArgument(colRef != null, "Cannot find rowkey column %s in cube %s", column, cubeDesc);
Preconditions.checkState(StringUtils.isNotEmpty(this.encoding));
Object[] encodingConf = DimensionEncoding.parseEncodingConf(this.encoding);
http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/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 dfd82de..f1a403d 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
@@ -25,7 +25,6 @@ import java.util.Map;
import java.util.Set;
import org.apache.commons.lang.ArrayUtils;
-import org.apache.kylin.dimension.DictionaryDimEnc;
import org.apache.kylin.metadata.model.TblColRef;
import com.fasterxml.jackson.annotation.JsonAutoDetect;
@@ -76,16 +75,16 @@ public class RowKeyDesc {
}
public void init(CubeDesc cubeDesc) {
-
setCubeDesc(cubeDesc);
- Map<String, TblColRef> colNameAbbr = cubeDesc.buildColumnNameAbbreviation();
-
- buildRowKey(colNameAbbr);
+ buildRowKey();
+ initColumnsNeedIndex();
+ }
+ private void initColumnsNeedIndex() {
int[] tmp = new int[100];
int x = 0;
for (int i = 0, n = rowkeyColumns.length; i < n; i++) {
- if ("true".equalsIgnoreCase(rowkeyColumns[i].getIndex()) && DictionaryDimEnc.ENCODING_NAME.equalsIgnoreCase(rowkeyColumns[i].getEncoding())) {
+ if ("true".equalsIgnoreCase(rowkeyColumns[i].getIndex()) && rowkeyColumns[i].isUsingDictionary()) {
tmp[x] = i;
x++;
}
@@ -103,13 +102,13 @@ public class RowKeyDesc {
return Objects.toStringHelper(this).add("RowKeyColumns", Arrays.toString(rowkeyColumns)).toString();
}
- private void buildRowKey(Map<String, TblColRef> colNameAbbr) {
- columnMap = new HashMap<TblColRef, RowKeyColDesc>();
+ private void buildRowKey() {
+ columnMap = new HashMap<>();
shardByColumns = new HashSet<>();
for (int i = 0; i < rowkeyColumns.length; i++) {
RowKeyColDesc rowKeyColDesc = rowkeyColumns[i];
- rowKeyColDesc.init(rowkeyColumns.length - i - 1, colNameAbbr, cubeDesc);
+ rowKeyColDesc.init(rowkeyColumns.length - i - 1, cubeDesc);
columnMap.put(rowKeyColDesc.getColRef(), rowKeyColDesc);
if (rowKeyColDesc.isShardBy()) {
http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/main/java/org/apache/kylin/cube/model/v1_4_0/CubeDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/v1_4_0/CubeDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/v1_4_0/CubeDesc.java
index 760b3e7..24c9ceb 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/v1_4_0/CubeDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/v1_4_0/CubeDesc.java
@@ -647,8 +647,6 @@ public class CubeDesc extends RootPersistentEntity {
return;
}
- TableDesc factTable = getFactTableDesc();
- List<TableDesc> lookups = getLookupTableDescs();
for (MeasureDesc m : measures) {
m.setName(m.getName().toUpperCase());
@@ -657,7 +655,7 @@ public class CubeDesc extends RootPersistentEntity {
}
FunctionDesc func = m.getFunction();
- func.init(factTable, lookups);
+ func.init(model);
allColumns.addAll(func.getParameter().getColRefs());
// // verify holistic count distinct as a dependent measure
http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/test/java/org/apache/kylin/cube/CubeDescTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/CubeDescTest.java b/core-cube/src/test/java/org/apache/kylin/cube/CubeDescTest.java
index 98cff43..db80025 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/CubeDescTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/CubeDescTest.java
@@ -26,7 +26,6 @@ import org.apache.kylin.common.util.JsonUtil;
import org.apache.kylin.common.util.LocalFileMetadataTestCase;
import org.apache.kylin.cube.model.CubeDesc;
import org.apache.kylin.cube.model.SelectRule;
-import org.apache.kylin.metadata.MetadataManager;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@@ -57,7 +56,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
@Test
public void testGoodInit() throws Exception {
CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
- cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+ cubeDesc.init(getTestConfig());
}
@Test
@@ -69,7 +68,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
String[] temp = null;
cubeDesc.getAggregationGroups().get(0).setIncludes(temp);
- cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+ cubeDesc.init(getTestConfig());
}
@Test
@@ -81,7 +80,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
SelectRule temp = null;
cubeDesc.getAggregationGroups().get(0).setSelectRule(temp);
- cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+ cubeDesc.init(getTestConfig());
}
@Test
@@ -93,7 +92,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
String[] temp = Arrays.asList(cubeDesc.getAggregationGroups().get(0).getIncludes()).subList(0, 3).toArray(new String[3]);
cubeDesc.getAggregationGroups().get(0).setIncludes(temp);
- cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+ cubeDesc.init(getTestConfig());
}
@Test
@@ -104,7 +103,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
try {
System.setProperty("kylin.cube.aggrgroup.max.combination", "8");
- cubeDesc.validate();
+ cubeDesc.validateAggregationGroups();
} finally {
System.clearProperty("kylin.cube.aggrgroup.max.combination");
}
@@ -115,7 +114,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
cubeDesc.getAggregationGroups().get(0).getSelectRule().mandatory_dims = new String[] { "seller_id", "META_CATEG_NAME" };
- cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+ cubeDesc.init(getTestConfig());
}
@Test
@@ -123,7 +122,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
cubeDesc.getAggregationGroups().get(0).getSelectRule().mandatory_dims = new String[] { "seller_id", "lstg_format_name" };
- cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+ cubeDesc.init(getTestConfig());
}
@Test
@@ -134,7 +133,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
cubeDesc.getAggregationGroups().get(0).getSelectRule().joint_dims = new String[][] { new String[] { "lstg_format_name" } };
- cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+ cubeDesc.init(getTestConfig());
}
@Test
@@ -145,7 +144,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
cubeDesc.getAggregationGroups().get(0).getSelectRule().joint_dims = new String[][] { new String[] { "META_CATEG_NAME", "CATEG_LVL2_NAME" } };
- cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+ cubeDesc.init(getTestConfig());
}
@Test
@@ -157,7 +156,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
cubeDesc.getAggregationGroups().get(0).getSelectRule().hierarchy_dims = new String[][] { new String[] { "META_CATEG_NAME", "CATEG_LVL2_NAME", "CATEG_LVL3_NAME" }, new String[] { "lstg_format_name", "lstg_site_id" } };
cubeDesc.getAggregationGroups().get(0).getSelectRule().joint_dims = new String[][] { new String[] { "META_CATEG_NAME", "lstg_format_name" } };
- cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+ cubeDesc.init(getTestConfig());
}
@Test
@@ -168,7 +167,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
cubeDesc.getAggregationGroups().get(0).getSelectRule().joint_dims = new String[][] { new String[] { "lstg_format_name", "lstg_site_id", "slr_segment_cd" }, new String[] { "lstg_format_name", "lstg_site_id", "leaf_categ_id" } };
- cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+ cubeDesc.init(getTestConfig());
}
@Test
@@ -179,7 +178,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
cubeDesc.getAggregationGroups().get(0).getSelectRule().hierarchy_dims = new String[][] { new String[] { "META_CATEG_NAME" } };
- cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+ cubeDesc.init(getTestConfig());
}
@Test
@@ -190,7 +189,7 @@ public class CubeDescTest extends LocalFileMetadataTestCase {
CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_with_slr_desc");
cubeDesc.getAggregationGroups().get(0).getSelectRule().hierarchy_dims = new String[][] { new String[] { "META_CATEG_NAME", "CATEG_LVL2_NAME", "CATEG_LVL3_NAME" }, new String[] { "META_CATEG_NAME", "CATEG_LVL2_NAME" } };
- cubeDesc.init(getTestConfig(), MetadataManager.getInstance(getTestConfig()).getAllTablesMap());
+ cubeDesc.init(getTestConfig());
}
@Test
http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java b/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java
index fdbbb2c..5492ad8 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/DictionaryRuleTest.java
@@ -32,7 +32,6 @@ import org.apache.kylin.cube.model.CubeDesc;
import org.apache.kylin.cube.model.DictionaryDesc;
import org.apache.kylin.cube.model.validation.ValidateContext;
import org.apache.kylin.dict.GlobalDictionaryBuilder;
-import org.apache.kylin.metadata.MetadataManager;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -42,13 +41,11 @@ import org.junit.Test;
*/
public class DictionaryRuleTest extends LocalFileMetadataTestCase {
private static KylinConfig config;
- private static MetadataManager metadataManager;
@Before
public void setUp() throws Exception {
this.createTestMetadata();
config = KylinConfig.getInstanceFromEnv();
- metadataManager = MetadataManager.getInstance(config);
}
@After
@@ -62,7 +59,7 @@ public class DictionaryRuleTest extends LocalFileMetadataTestCase {
for (File f : new File(LocalFileMetadataTestCase.LOCALMETA_TEST_DATA + "/cube_desc/").listFiles()) {
CubeDesc desc = JsonUtil.readValue(new FileInputStream(f), CubeDesc.class);
- desc.init(config, metadataManager.getAllTablesMap());
+ desc.init(config);
ValidateContext vContext = new ValidateContext();
rule.validate(desc, vContext);
vContext.print(System.out);
@@ -99,7 +96,7 @@ public class DictionaryRuleTest extends LocalFileMetadataTestCase {
desc.getDictionaries().add(dictDesc);
}
- desc.init(config, metadataManager.getAllTablesMap());
+ desc.init(config);
ValidateContext vContext = new ValidateContext();
rule.validate(desc, vContext);
vContext.print(System.out);
http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/FunctionRuleTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/FunctionRuleTest.java b/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/FunctionRuleTest.java
index e041080..a3456c4 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/FunctionRuleTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/model/validation/rule/FunctionRuleTest.java
@@ -18,33 +18,30 @@
package org.apache.kylin.cube.model.validation.rule;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.util.JsonUtil;
import org.apache.kylin.common.util.LocalFileMetadataTestCase;
import org.apache.kylin.cube.model.CubeDesc;
import org.apache.kylin.cube.model.validation.ValidateContext;
-import org.apache.kylin.metadata.MetadataManager;
import org.apache.kylin.metadata.model.MeasureDesc;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
public class FunctionRuleTest extends LocalFileMetadataTestCase {
private static KylinConfig config;
- private static MetadataManager metadataManager;
@Before
public void setUp() throws Exception {
this.createTestMetadata();
config = KylinConfig.getInstanceFromEnv();
- metadataManager = MetadataManager.getInstance(config);
}
@After
@@ -58,7 +55,7 @@ public class FunctionRuleTest extends LocalFileMetadataTestCase {
File f = new File(LocalFileMetadataTestCase.LOCALMETA_TEST_DATA + "/cube_desc/ssb.json");
CubeDesc desc = JsonUtil.readValue(new FileInputStream(f), CubeDesc.class);
- desc.init(config, metadataManager.getAllTablesMap());
+ desc.init(config);
ValidateContext vContext = new ValidateContext();
rule.validate(desc, vContext);
vContext.print(System.out);
@@ -75,7 +72,7 @@ public class FunctionRuleTest extends LocalFileMetadataTestCase {
MeasureDesc measureDescDuplicated = desc.getMeasures().get(1);
desc.getMeasures().add(measureDescDuplicated);
- desc.init(config, metadataManager.getAllTablesMap());
+ desc.init(config);
ValidateContext vContext = new ValidateContext();
rule.validate(desc, vContext);
http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
index 36fff5c..a2c5756 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.LinkedHashMap;
-import java.util.List;
import java.util.Set;
import org.apache.kylin.measure.MeasureType;
@@ -75,7 +74,7 @@ public class FunctionDesc {
private MeasureType<?> measureType;
private boolean isDimensionAsMetric = false;
- public void init(TableDesc factTable, List<TableDesc> lookupTables) {
+ public void init(DataModelDesc model) {
expression = expression.toUpperCase();
returnDataType = DataType.getType(returnType);
@@ -86,8 +85,7 @@ public class FunctionDesc {
ArrayList<TblColRef> colRefs = Lists.newArrayList();
for (ParameterDesc p = parameter; p != null; p = p.getNextParameter()) {
if (p.isColumnType()) {
- ColumnDesc sourceColumn = findColumn(factTable, lookupTables, p.getValue());
- TblColRef colRef = new TblColRef(sourceColumn);
+ TblColRef colRef = model.findColumn(p.getValue());
colRefs.add(colRef);
}
}
@@ -95,21 +93,6 @@ public class FunctionDesc {
parameter.setColRefs(colRefs);
}
- private ColumnDesc findColumn(TableDesc factTable, List<TableDesc> lookups, String columnName) {
- ColumnDesc ret = factTable.findColumnByName(columnName);
- if (ret != null) {
- return ret;
- }
-
- for (TableDesc lookup : lookups) {
- ret = lookup.findColumnByName(columnName);
- if (ret != null) {
- return ret;
- }
- }
- throw new IllegalStateException("Column is not found in any table from the model: " + columnName);
- }
-
private void reInitMeasureType() {
if (isDimensionAsMetric && isCountDistinct()) {
// create DimCountDis
http://git-wip-us.apache.org/repos/asf/kylin/blob/b1a96509/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
index a5a85fa..b883df2 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/ExtendCubeToHybridCLI.java
@@ -170,7 +170,7 @@ public class ExtendCubeToHybridCLI {
CubeDesc newCubeDesc = CubeDesc.getCopyOf(cubeDesc);
newCubeDesc.setName(newCubeDescName);
newCubeDesc.updateRandomUuid();
- newCubeDesc.init(kylinConfig, metadataManager.getAllTablesMap());
+ newCubeDesc.init(kylinConfig);
newCubeDesc.setPartitionDateEnd(partitionDate);
newCubeDesc.calculateSignature();
cubeDescManager.createCubeDesc(newCubeDesc);