You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by ma...@apache.org on 2015/09/17 08:06:52 UTC

[1/4] incubator-kylin git commit: KYLIN-1028 force save dict when migration

Repository: incubator-kylin
Updated Branches:
  refs/heads/2.x-staging 4456bb1dd -> d2cdda730


KYLIN-1028 force save dict when migration


Project: http://git-wip-us.apache.org/repos/asf/incubator-kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-kylin/commit/d2cdda73
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/d2cdda73
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/d2cdda73

Branch: refs/heads/2.x-staging
Commit: d2cdda730cb7bfc8621e065ca5ea7e1ec470f000
Parents: cbf7205
Author: honma <ho...@ebay.com>
Authored: Thu Sep 17 14:08:03 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Thu Sep 17 14:09:09 2015 +0800

----------------------------------------------------------------------
 .../storage/hbase/util/CubeMigrationCLI.java    | 31 +++++---------------
 1 file changed, 7 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/d2cdda73/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
index c4a54cb..5708514 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
@@ -56,6 +56,8 @@ import org.apache.kylin.metadata.realization.RealizationType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Preconditions;
+
 /**
  * <p/>
  * This tool serves for the purpose of migrating cubes. e.g. upgrade cube from
@@ -317,31 +319,12 @@ public class CubeMigrationCLI {
                 DictionaryManager srcDicMgr = DictionaryManager.getInstance(srcConfig);
                 DictionaryInfo dictSrc = srcDicMgr.getDictionaryInfo(item);
 
-                long ts = dictSrc.getLastModified();
-                dictSrc.setLastModified(0);//to avoid resource store write conflict
-                DictionaryInfo dictSaved = dstDictMgr.trySaveNewDict(dictSrc.getDictionaryObject(), dictSrc);
-                dictSrc.setLastModified(ts);
+                //trySaveNewDict will return a super set of dictSrc,
+                //which is not expected in migration case
+                DictionaryInfo dictSaved = dstDictMgr.forceSave(dictSrc.getDictionaryObject(), dictSrc);
 
-                if (dictSaved == dictSrc) {
-                    //no dup found, already saved to dest
-                    logger.info("Item " + item + " is copied");
-                } else {
-                    //dictSrc is rejected because of duplication
-                    //modify cube's dictionary path
-                    String cubeName = (String) opt.params[1];
-                    String cubeResPath = CubeInstance.concatResourcePath(cubeName);
-                    Serializer<CubeInstance> cubeSerializer = new JsonSerializer<CubeInstance>(CubeInstance.class);
-                    CubeInstance cube = dstStore.getResource(cubeResPath, CubeInstance.class, cubeSerializer);
-                    for (CubeSegment segment : cube.getSegments()) {
-                        for (Map.Entry<String, String> entry : segment.getDictionaries().entrySet()) {
-                            if (entry.getValue().equalsIgnoreCase(item)) {
-                                entry.setValue(dictSaved.getResourcePath());
-                            }
-                        }
-                    }
-                    dstStore.putResource(cubeResPath, cube, cubeSerializer);
-                    logger.info("Item " + item + " is dup, instead " + dictSaved.getResourcePath() + " is reused");
-                }
+                Preconditions.checkState(dictSaved == dictSrc);
+                logger.info("Item " + item + " is copied");
 
             } else if (item.toLowerCase().endsWith(".snapshot")) {
                 SnapshotManager dstSnapMgr = SnapshotManager.getInstance(dstConfig);


[3/4] incubator-kylin git commit: KYLIN-1029 saving dict skip checking ts

Posted by ma...@apache.org.
KYLIN-1029 saving dict skip checking ts


Project: http://git-wip-us.apache.org/repos/asf/incubator-kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-kylin/commit/cbf72050
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/cbf72050
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/cbf72050

Branch: refs/heads/2.x-staging
Commit: cbf72050be28378f7485e44a06350f909a0fae84
Parents: 7e3f4ca
Author: honma <ho...@ebay.com>
Authored: Thu Sep 17 14:07:40 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Thu Sep 17 14:09:09 2015 +0800

----------------------------------------------------------------------
 .../kylin/common/persistence/ResourceStore.java |  2 +-
 .../apache/kylin/dict/DictionaryManager.java    | 34 +++++++++++++++++---
 2 files changed, 31 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/cbf72050/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java b/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
index b6ba550..29e2345 100644
--- a/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
+++ b/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
@@ -191,7 +191,7 @@ abstract public class ResourceStore {
      */
     final public void putResource(String resPath, InputStream content, long ts) throws IOException {
         resPath = norm(resPath);
-        logger.debug("Saving resource " + resPath + " (Store " + kylinConfig.getMetadataUrl() + ")");
+        logger.debug("Directly saving resource " + resPath + " (Store " + kylinConfig.getMetadataUrl() + ")");
         putResourceImpl(resPath, content, ts);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/cbf72050/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
index c4b6ef0..981e19a 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryManager.java
@@ -18,6 +18,9 @@
 
 package org.apache.kylin.dict;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -111,14 +114,22 @@ public class DictionaryManager {
     }
 
     /**
+     * Save the dictionary as it is.
+     * More often you should consider using its alternative trySaveNewDict to save dict space
+     */
+    public DictionaryInfo forceSave(Dictionary<?> newDict, DictionaryInfo newDictInfo) throws IOException {
+        initDictInfo(newDict, newDictInfo);
+        logger.info("force to save dict directly");
+        return saveNewDict(newDictInfo);
+    }
+
+    /**
      * @return may return another dict that is a super set of the input
      * @throws IOException
      */
     public DictionaryInfo trySaveNewDict(Dictionary<?> newDict, DictionaryInfo newDictInfo) throws IOException {
 
-        newDictInfo.setCardinality(newDict.getSize());
-        newDictInfo.setDictionaryObject(newDict);
-        newDictInfo.setDictionaryClass(newDict.getClass().getName());
+        initDictInfo(newDict, newDictInfo);
 
         DictionaryInfo largestDictInfo = findLargestDictInfo(newDictInfo);
         if (largestDictInfo != null) {
@@ -140,6 +151,12 @@ public class DictionaryManager {
         }
     }
 
+    private void initDictInfo(Dictionary<?> newDict, DictionaryInfo newDictInfo) {
+        newDictInfo.setCardinality(newDict.getSize());
+        newDictInfo.setDictionaryObject(newDict);
+        newDictInfo.setDictionaryClass(newDict.getClass().getName());
+    }
+
     private DictionaryInfo saveNewDict(DictionaryInfo newDictInfo) throws IOException {
 
         save(newDictInfo);
@@ -324,7 +341,16 @@ public class DictionaryManager {
         ResourceStore store = MetadataManager.getInstance(config).getStore();
         String path = dict.getResourcePath();
         logger.info("Saving dictionary at " + path);
-        store.putResource(path, dict, DictionaryInfoSerializer.FULL_SERIALIZER);
+
+        ByteArrayOutputStream buf = new ByteArrayOutputStream();
+        DataOutputStream dout = new DataOutputStream(buf);
+        DictionaryInfoSerializer.FULL_SERIALIZER.serialize(dict, dout);
+        dout.close();
+        buf.close();
+
+        ByteArrayInputStream inputStream = new ByteArrayInputStream(buf.toByteArray());
+        store.putResource(path, inputStream, System.currentTimeMillis());
+        inputStream.close();
     }
 
     DictionaryInfo load(String resourcePath, boolean loadDictObj) throws IOException {


[2/4] incubator-kylin git commit: KYLIN-1027 init credentials manually for HBASE token

Posted by ma...@apache.org.
KYLIN-1027 init credentials manually for HBASE token


Project: http://git-wip-us.apache.org/repos/asf/incubator-kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-kylin/commit/7e3f4ca8
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/7e3f4ca8
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/7e3f4ca8

Branch: refs/heads/2.x-staging
Commit: 7e3f4ca8eb3022291a97a8682901a184291e8b39
Parents: 3a6ad20
Author: honma <ho...@ebay.com>
Authored: Wed Sep 16 17:37:50 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Thu Sep 17 14:09:09 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/storage/hbase/steps/HBaseMROutput2.java   | 6 +++---
 .../kylin/storage/hbase/steps/HBaseMROutput2Transition.java    | 1 +
 2 files changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e3f4ca8/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2.java
index 774fe2a..f99ddb5 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2.java
@@ -129,7 +129,7 @@ public class HBaseMROutput2 implements IMROutput2 {
     @SuppressWarnings({ "rawtypes", "unchecked" })
     private static class HBaseInputFormat implements IMRStorageInputFormat {
         final CubeSegment seg;
-        
+
         final RowValueDecoder[] rowValueDecoders;
         final ByteArrayWritable parsedKey;
         final Object[] parsedValue;
@@ -137,7 +137,7 @@ public class HBaseMROutput2 implements IMROutput2 {
 
         public HBaseInputFormat(CubeSegment seg) {
             this.seg = seg;
-            
+
             List<RowValueDecoder> valueDecoderList = Lists.newArrayList();
             List<MeasureDesc> measuresDescs = Lists.newArrayList();
             for (HBaseColumnFamilyDesc cfDesc : seg.getCubeDesc().getHBaseMapping().getColumnFamily()) {
@@ -170,7 +170,7 @@ public class HBaseMROutput2 implements IMROutput2 {
             }
 
             TableMapReduceUtil.initTableMapperJob(scans, (Class<? extends TableMapper>) mapperClz, outputKeyClz, outputValueClz, job);
-TableMapReduceUtil.initCredentials(job);
+            TableMapReduceUtil.initCredentials(job);
 
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/7e3f4ca8/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java
index 237f0c9..00cecf1 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2Transition.java
@@ -184,6 +184,7 @@ public class HBaseMROutput2Transition implements IMROutput2 {
                     scans.add(scan);
                 }
                 TableMapReduceUtil.initTableMapperJob(scans, (Class<? extends TableMapper>) mapperClz, outputKeyClz, outputValueClz, job);
+                TableMapReduceUtil.initCredentials(job);
             }
         }
 


[4/4] incubator-kylin git commit: KYLIN-1028 Keep Migration tool working for v1 cubes and v2 cubes on 2.x-staging

Posted by ma...@apache.org.
KYLIN-1028 Keep Migration tool working for v1 cubes and v2 cubes on 2.x-staging


Project: http://git-wip-us.apache.org/repos/asf/incubator-kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-kylin/commit/3a6ad20b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-kylin/tree/3a6ad20b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-kylin/diff/3a6ad20b

Branch: refs/heads/2.x-staging
Commit: 3a6ad20bf253606ca8e30f6f6005145c91084134
Parents: 4456bb1
Author: honma <ho...@ebay.com>
Authored: Wed Sep 16 17:37:26 2015 +0800
Committer: honma <ho...@ebay.com>
Committed: Thu Sep 17 14:09:09 2015 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/KylinConfig.java    |   2 +-
 .../storage/hbase/steps/HBaseMROutput2.java     |   3 +
 .../storage/hbase/util/CubeMigrationCLI.java    |  19 +++-
 webapp/app/js/model/cubeDescModel.js            | 104 ++++++++++---------
 4 files changed, 71 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3a6ad20b/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java b/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
index db213f7..43b8c4d 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfig.java
@@ -502,7 +502,7 @@ public class KylinConfig implements Serializable {
     }
 
     public String getHbaseDefaultCompressionCodec() {
-        return getOptional(HTABLE_DEFAULT_COMPRESSION_CODEC);
+        return getOptional(HTABLE_DEFAULT_COMPRESSION_CODEC,"");
     }
 
     public boolean isHiveKeepFlatTable() {

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3a6ad20b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2.java
index b468009..774fe2a 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMROutput2.java
@@ -168,7 +168,10 @@ public class HBaseMROutput2 implements IMROutput2 {
                 scan.setAttribute(Scan.SCAN_ATTRIBUTES_TABLE_NAME, Bytes.toBytes(htable));
                 scans.add(scan);
             }
+
             TableMapReduceUtil.initTableMapperJob(scans, (Class<? extends TableMapper>) mapperClz, outputKeyClz, outputValueClz, job);
+TableMapReduceUtil.initCredentials(job);
+
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3a6ad20b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
index 6dfa1d6..c4a54cb 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CubeMigrationCLI.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -45,6 +46,7 @@ import org.apache.kylin.dict.lookup.SnapshotManager;
 import org.apache.kylin.dict.lookup.SnapshotTable;
 import org.apache.kylin.job.JobInstance;
 import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.model.IEngineAware;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.project.ProjectInstance;
@@ -87,7 +89,6 @@ public class CubeMigrationCLI {
     private static void usage() {
         System.out.println("Usage: CubeMigrationCLI srcKylinConfigUri dstKylinConfigUri cubeName projectName overwriteIfExists realExecute");
         System.out.println(" srcKylinConfigUri: The KylinConfig of the cube’s source \n" + "dstKylinConfigUri: The KylinConfig of the cube’s new home \n" + "cubeName: the name of cube to be migrated. \n" + "projectName: The target project in the target environment.(Make sure it exist) \n" + "overwriteIfExists: overwrite cube if it already exists in the target environment. \n" + "realExecute: if false, just print the operations to take, if true, do the real migration. \n");
-
     }
 
     public static void moveCube(KylinConfig srcCfg, KylinConfig dstCfg, String cubeName, String projectName, String overwriteIfExists, String realExecute) throws IOException, InterruptedException {
@@ -132,7 +133,6 @@ public class CubeMigrationCLI {
     }
 
     public static void moveCube(String srcCfgUri, String dstCfgUri, String cubeName, String projectName, String overwriteIfExists, String realExecute) throws IOException, InterruptedException {
-
         moveCube(KylinConfig.createInstanceFromUri(srcCfgUri), KylinConfig.createInstanceFromUri(dstCfgUri), cubeName, projectName, overwriteIfExists, realExecute);
     }
 
@@ -143,8 +143,8 @@ public class CubeMigrationCLI {
         logger.info("src metadata url is " + srcMetadataUrl);
         logger.info("dst metadata url is " + dstMetadataUrl);
 
-        int srcIndex = srcMetadataUrl.toLowerCase().indexOf("hbase:");
-        int dstIndex = dstMetadataUrl.toLowerCase().indexOf("hbase:");
+        int srcIndex = srcMetadataUrl.toLowerCase().indexOf("hbase");
+        int dstIndex = dstMetadataUrl.toLowerCase().indexOf("hbase");
         if (srcIndex < 0 || dstIndex < 0)
             throw new IllegalStateException("Both metadata urls should be hbase metadata url");
 
@@ -162,6 +162,11 @@ public class CubeMigrationCLI {
         for (CubeSegment segment : cube.getSegments()) {
 
             String jobUuid = segment.getLastBuildJobID();
+
+            if (StringUtils.isEmpty(jobUuid)) {
+                //segments build from streaming does not have hdfs working dir
+                continue;
+            }
             String src = JobInstance.getJobWorkingDir(jobUuid, srcConfig.getHdfsWorkingDirectory());
             String tgt = JobInstance.getJobWorkingDir(jobUuid, dstConfig.getHdfsWorkingDirectory());
 
@@ -216,7 +221,11 @@ public class CubeMigrationCLI {
         for (CubeSegment segment : cube.getSegments()) {
             dictAndSnapshot.addAll(segment.getSnapshotPaths());
             dictAndSnapshot.addAll(segment.getDictionaryPaths());
-            metaResource.add(segment.getStatisticsResourcePath());
+
+            if (cube.getDescriptor().getEngineType() == IEngineAware.ID_MR_V2) {
+                //only V2 has this
+                metaResource.add(segment.getStatisticsResourcePath());
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-kylin/blob/3a6ad20b/webapp/app/js/model/cubeDescModel.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/model/cubeDescModel.js b/webapp/app/js/model/cubeDescModel.js
index 86c8444..3376bfc 100644
--- a/webapp/app/js/model/cubeDescModel.js
+++ b/webapp/app/js/model/cubeDescModel.js
@@ -14,62 +14,64 @@
  * 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.
-*/
+ */
 
-KylinApp.service('CubeDescModel',function(){
+KylinApp.service('CubeDescModel', function () {
 
-    this.cubeMetaFrame = {};
+  this.cubeMetaFrame = {};
 
-    //
-    this.createNew = function () {
-            var cubeMeta = {
-                "name": "",
-                "model_name": "",
-                "description": "",
-                "dimensions": [],
-                "measures": [
-                    {   "id": 1,
-                        "name": "_COUNT_",
-                        "function": {
-                            "expression": "COUNT",
-                            "returntype": "bigint",
-                            "parameter": {
-                                "type": "constant",
-                                "value": "1"
-                            }
-                        }
-                    }
-                ],
-                "rowkey": {
-                    "rowkey_columns": [],
-                    "aggregation_groups": []
-                },
-                "notify_list": [],
-                "hbase_mapping": {
-                    "column_family": []
-                },
-               "retention_range":"0",
-               "auto_merge_time_ranges":[604800000,2419200000]
-            };
-
-            return cubeMeta;
-        };
+  //
+  this.createNew = function () {
+    var cubeMeta = {
+      "name": "",
+      "model_name": "",
+      "description": "",
+      "dimensions": [],
+      "measures": [
+        {
+          "id": 1,
+          "name": "_COUNT_",
+          "function": {
+            "expression": "COUNT",
+            "returntype": "bigint",
+            "parameter": {
+              "type": "constant",
+              "value": "1"
+            }
+          }
+        }
+      ],
+      "rowkey": {
+        "rowkey_columns": [],
+        "aggregation_groups": []
+      },
+      "notify_list": [],
+      "hbase_mapping": {
+        "column_family": []
+      },
+      "retention_range": "0",
+      "auto_merge_time_ranges": [604800000, 2419200000],
+      "engine_type": 2
+    };
 
-        this.createMeasure = function (){
-            var measure = {
-                "id": "",
-                "name": "",
-                "function": {
-                    "expression": "",
-                    "returntype": "",
-                    "parameter": {
-                        "type": "",
-                        "value": ""
-                    }
-                }
-            };
+    return cubeMeta;
+  };
 
-            return measure;
+  this.createMeasure = function () {
+    var measure = {
+      "id": "",
+      "name": "",
+      "function": {
+        "expression": "",
+        "returntype": "",
+        "parameter": {
+          "type": "",
+          "value": ""
         }
+      }
+    };
+
+    return measure;
+  }
 
 })