[12/13] kylin git commit: KYLIN-1528 Create a branch for v1.5 with HBase 1.x API

2016-10-30 Thread mahongbin
http://git-wip-us.apache.org/repos/asf/kylin/blob/d1382776/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
--
diff --git 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
index 5b2441c..2f7e164 100644
--- 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
+++ 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/MergeGCStep.java
@@ -24,11 +24,11 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.job.exception.ExecuteException;
 import org.apache.kylin.job.execution.AbstractExecutable;
 import org.apache.kylin.job.execution.ExecutableContext;
@@ -69,19 +69,20 @@ public class MergeGCStep extends AbstractExecutable {
 List oldTables = getOldHTables();
 if (oldTables != null && oldTables.size() > 0) {
 String metadataUrlPrefix = 
KylinConfig.getInstanceFromEnv().getMetadataUrlPrefix();
-Configuration conf = 
HBaseConnection.getCurrentHBaseConfiguration();
-HBaseAdmin admin = null;
+Admin admin = null;
 try {
-admin = new HBaseAdmin(conf);
+Connection conn = 
HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+admin = conn.getAdmin();
+
 for (String table : oldTables) {
-if (admin.tableExists(table)) {
-HTableDescriptor tableDescriptor = 
admin.getTableDescriptor(Bytes.toBytes(table));
+if (admin.tableExists(TableName.valueOf(table))) {
+HTableDescriptor tableDescriptor = 
admin.getTableDescriptor(TableName.valueOf((table)));
 String host = 
tableDescriptor.getValue(IRealizationConstants.HTableTag);
 if (metadataUrlPrefix.equalsIgnoreCase(host)) {
-if (admin.isTableEnabled(table)) {
-admin.disableTable(table);
+if 
(admin.isTableEnabled(TableName.valueOf(table))) {
+admin.disableTable(TableName.valueOf(table));
 }
-admin.deleteTable(table);
+admin.deleteTable(TableName.valueOf(table));
 logger.debug("Dropped htable: " + table);
 output.append("HBase table " + table + " is 
dropped. \n");
 } else {

http://git-wip-us.apache.org/repos/asf/kylin/blob/d1382776/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
--
diff --git 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
index a150607..56f867a 100644
--- 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
+++ 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/util/CleanHtableCLI.java
@@ -21,9 +21,11 @@ package org.apache.kylin.storage.hbase.util;
 import java.io.IOException;
 
 import org.apache.commons.cli.Options;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.AbstractApplication;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.apache.kylin.metadata.realization.IRealizationConstants;
@@ -38,8 +40,8 @@ public class CleanHtableCLI extends AbstractApplication {
 protected static final Logger logger = 
LoggerFactory.getLogger(CleanHtableCLI.class);
 
 private void clean() throws IOException {
-Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-HBaseAdmin hbaseAdmin = new HBaseAdmin(conf);
+Connection conn = 
HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl());
+Admin hbaseAdmin = conn.getAdmin();
 
 for (HTableDescriptor descriptor : hbaseAdmin.listTables()) {
 String name = 

[05/13] kylin git commit: minor, remove kylin.dictionary.max.cardinality

2016-10-30 Thread mahongbin
minor, remove kylin.dictionary.max.cardinality


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

Branch: refs/heads/yang21-hbase1.x
Commit: 2d87faec4d04e842ea73bb0493209b661305a4ee
Parents: b502a17
Author: Li Yang 
Authored: Thu Oct 27 14:34:40 2016 +0800
Committer: Li Yang 
Committed: Thu Oct 27 14:35:43 2016 +0800

--
 .../java/org/apache/kylin/common/KylinConfigBase.java |  4 
 .../org/apache/kylin/dict/DictionaryGenerator.java| 14 --
 2 files changed, 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/2d87faec/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
--
diff --git 
a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java 
b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index 5d92aef..53074fd 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -448,10 +448,6 @@ abstract public class KylinConfigBase implements 
Serializable {
 return getPropertiesByPrefix("kylin.cube.measure.customMeasureType.");
 }
 
-public int getDictionaryMaxCardinality() {
-return 
Integer.parseInt(getOptional("kylin.dictionary.max.cardinality", "3000"));
-}
-
 public int getTableSnapshotMaxMB() {
 return Integer.parseInt(getOptional("kylin.table.snapshot.max_mb", 
"300"));
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/2d87faec/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
--
diff --git 
a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java 
b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
index ba848c6..0adf40e 100644
--- 
a/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
+++ 
b/core-dictionary/src/main/java/org/apache/kylin/dict/DictionaryGenerator.java
@@ -25,7 +25,6 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.metadata.datatype.DataType;
@@ -40,20 +39,10 @@ import com.google.common.base.Preconditions;
 @SuppressWarnings({ "rawtypes", "unchecked" })
 public class DictionaryGenerator {
 
-private static final int DICT_MAX_CARDINALITY = 
getDictionaryMaxCardinality();
-
 private static final Logger logger = 
LoggerFactory.getLogger(DictionaryGenerator.class);
 
 private static final String[] DATE_PATTERNS = new String[] { "-MM-dd", 
"MMdd" };
 
-private static int getDictionaryMaxCardinality() {
-try {
-return 
KylinConfig.getInstanceFromEnv().getDictionaryMaxCardinality();
-} catch (Throwable e) {
-return 3000; // some test case does not have KylinConfig setup 
properly
-}
-}
-
 public static Dictionary buildDictionary(DataType dataType, 
IDictionaryValueEnumerator valueEnumerator) throws IOException {
 Preconditions.checkNotNull(dataType, "dataType cannot be null");
 
@@ -92,9 +81,6 @@ public class DictionaryGenerator {
 logger.debug("Dictionary cardinality: " + dict.getSize());
 logger.debug("Dictionary builder class: " + 
builder.getClass().getName());
 logger.debug("Dictionary class: " + dict.getClass().getName());
-if (dict instanceof TrieDictionary && dict.getSize() > 
DICT_MAX_CARDINALITY) {
-throw new IllegalArgumentException("Too high cardinality is not 
suitable for dictionary -- cardinality: " + dict.getSize());
-}
 return dict;
 }
 



[04/13] kylin git commit: port KYLIN-2012 to new interface introduced in KYLIN-2125

2016-10-30 Thread mahongbin
port KYLIN-2012 to new interface introduced in KYLIN-2125


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

Branch: refs/heads/yang21-hbase1.x
Commit: b502a17414468652b75f84994fb371a767784a47
Parents: 6fc1c86
Author: Hongbin Ma 
Authored: Wed Oct 26 14:04:56 2016 +0800
Committer: Hongbin Ma 
Committed: Wed Oct 26 18:49:34 2016 +0800

--
 .../java/org/apache/kylin/job/DeployUtil.java   |  5 ++-
 .../kylin/source/hive/BeelineHiveClient.java|  6 +--
 .../source/hive/HiveSourceTableLoader.java  | 32 +++
 .../apache/kylin/source/hive/SchemaChecker.java | 41 
 4 files changed, 38 insertions(+), 46 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/b502a174/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
--
diff --git a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java 
b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
index 8c64f91..0734f4f 100644
--- a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -38,8 +38,9 @@ import org.apache.kylin.metadata.MetadataManager;
 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.hive.HiveClient;
+import org.apache.kylin.source.hive.HiveClientFactory;
 import org.apache.kylin.source.hive.HiveCmdBuilder;
+import org.apache.kylin.source.hive.IHiveClient;
 import org.apache.kylin.source.kafka.TimedJsonStreamParser;
 import org.apache.maven.model.Model;
 import org.apache.maven.model.io.xpp3.MavenXpp3Reader;
@@ -205,7 +206,7 @@ public class DeployUtil {
 String tableFileDir = temp.getParent();
 temp.delete();
 
-HiveClient hiveClient = new HiveClient();
+IHiveClient hiveClient = HiveClientFactory.getHiveClient();
 // create hive tables
 hiveClient.executeHQL("CREATE DATABASE IF NOT EXISTS EDW");
 
hiveClient.executeHQL(generateCreateTableHql(metaMgr.getTableDesc(TABLE_CAL_DT.toUpperCase(;

http://git-wip-us.apache.org/repos/asf/kylin/blob/b502a174/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
index 0fbc39b..b027f2e 100644
--- 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
+++ 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
@@ -205,9 +205,9 @@ public class BeelineHiveClient implements IHiveClient {
 
 public static void main(String[] args) throws SQLException {
 
-//BeelineHiveClient loader = new BeelineHiveClient("-n root --hiveconf 
hive.security.authorization.sqlstd.confwhitelist.append='mapreduce.job.*|dfs.*' 
-u 'jdbc:hive2://sandbox:1'");
-BeelineHiveClient loader = new 
BeelineHiveClient(StringUtils.join(args, " "));
-HiveTableMeta hiveTableMeta = loader.getHiveTableMeta("default", 
"events");
+BeelineHiveClient loader = new BeelineHiveClient("-n root --hiveconf 
hive.security.authorization.sqlstd.confwhitelist.append='mapreduce.job.*|dfs.*' 
-u 'jdbc:hive2://sandbox:1'");
+//BeelineHiveClient loader = new 
BeelineHiveClient(StringUtils.join(args, " "));
+HiveTableMeta hiveTableMeta = loader.getHiveTableMeta("default", 
"test_kylin_fact_part");
 System.out.println(hiveTableMeta);
 loader.close();
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/b502a174/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
 
b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
index 346d278..1335ec3 100644
--- 
a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
+++ 
b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
@@ -26,6 +26,7 @@ import java.util.Set;
 import java.util.UUID;
 
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.metadata.MetadataConstants;
 import 

[11/13] kylin git commit: KYLIN-2137 Failed to run mr job when user put a kafka jar in hive's lib folder

2016-10-30 Thread mahongbin
KYLIN-2137 Failed to run mr job when user put a kafka jar in hive's lib folder


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

Branch: refs/heads/yang21-hbase1.x
Commit: 066a38e3b7b1c2345f9f12106173eccbcfbfb0e7
Parents: 9e1e70a
Author: shaofengshi 
Authored: Fri Oct 28 18:15:33 2016 +0800
Committer: shaofengshi 
Committed: Fri Oct 28 22:09:13 2016 +0800

--
 .../apache/kylin/common/util/StringUtil.java| 13 +
 .../engine/mr/common/AbstractHadoopJob.java | 28 +++-
 2 files changed, 23 insertions(+), 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/066a38e3/core-common/src/main/java/org/apache/kylin/common/util/StringUtil.java
--
diff --git 
a/core-common/src/main/java/org/apache/kylin/common/util/StringUtil.java 
b/core-common/src/main/java/org/apache/kylin/common/util/StringUtil.java
index 15872aa..b336e4b 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/StringUtil.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/StringUtil.java
@@ -124,4 +124,17 @@ public class StringUtil {
 return result;
 }
 
+public static void appendWithSeparator(StringBuilder src, String append) {
+if (src == null) {
+throw new IllegalArgumentException();
+}
+if (src.length() > 0 && src.toString().endsWith(",") == false) {
+src.append(",");
+}
+
+if (StringUtils.isBlank(append) == false) {
+src.append(append);
+}
+}
+
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/066a38e3/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
--
diff --git 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
index af2ed9f..b77b6b0 100644
--- 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
+++ 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
@@ -61,6 +61,7 @@ import org.apache.kylin.common.persistence.ResourceStore;
 import org.apache.kylin.common.util.CliCommandExecutor;
 import org.apache.kylin.common.util.OptionsHelper;
 import org.apache.kylin.common.util.StringSplitter;
+import org.apache.kylin.common.util.StringUtil;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeSegment;
 import org.apache.kylin.engine.mr.HadoopUtil;
@@ -202,24 +203,23 @@ public abstract class AbstractHadoopJob extends 
Configured implements Tool {
 String filteredHive = 
filterKylinHiveDependency(kylinHiveDependency);
 logger.info("Hive Dependencies After Filtered: " + filteredHive);
 
-if (kylinDependency.length() > 0)
-kylinDependency.append(",");
-kylinDependency.append(filteredHive);
+StringUtil.appendWithSeparator(kylinDependency, filteredHive);
 } else {
 
 logger.info("No hive dependency jars set in the environment, will 
find them from jvm:");
 
 try {
 String hiveExecJarPath = 
ClassUtil.findContainingJar(Class.forName("org.apache.hadoop.hive.ql.Driver"));
-kylinDependency.append(hiveExecJarPath).append(",");
+
+StringUtil.appendWithSeparator(kylinDependency, 
hiveExecJarPath);
 logger.info("hive-exec jar file: " + hiveExecJarPath);
 
 String hiveHCatJarPath = 
ClassUtil.findContainingJar(Class.forName("org.apache.hive.hcatalog.mapreduce.HCatInputFormat"));
-kylinDependency.append(hiveHCatJarPath).append(",");
+StringUtil.appendWithSeparator(kylinDependency, 
hiveHCatJarPath);
 logger.info("hive-catalog jar file: " + hiveHCatJarPath);
 
 String hiveMetaStoreJarPath = 
ClassUtil.findContainingJar(Class.forName("org.apache.hadoop.hive.metastore.api.Table"));
-kylinDependency.append(hiveMetaStoreJarPath).append(",");
+StringUtil.appendWithSeparator(kylinDependency, 
hiveMetaStoreJarPath);
 logger.info("hive-metastore jar file: " + 
hiveMetaStoreJarPath);
 } catch (ClassNotFoundException e) {
 logger.error("Cannot found hive dependency jars: " + e);
@@ -229,19 +229,13 @@ public abstract class AbstractHadoopJob extends 
Configured implements Tool {
 // for hive dependencies
 

[02/13] kylin git commit: KYLIN-2012 adjust rules for LookupTable & remove dead methods

2016-10-30 Thread mahongbin
KYLIN-2012 adjust rules for LookupTable & remove dead methods


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

Branch: refs/heads/yang21-hbase1.x
Commit: d5565f775fa0e907a5a2c0bc33c75a2ece91ae31
Parents: f56c3c4
Author: gaodayue 
Authored: Thu Oct 13 16:56:05 2016 +0800
Committer: Hongbin Ma 
Committed: Wed Oct 26 18:39:05 2016 +0800

--
 .../apache/kylin/metadata/MetadataManager.java  | 30 
 .../apache/kylin/source/hive/SchemaChecker.java | 27 ++
 2 files changed, 21 insertions(+), 36 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/d5565f77/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
--
diff --git 
a/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java 
b/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
index 8d77131..fd0a3c3 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/MetadataManager.java
@@ -461,11 +461,6 @@ public class MetadataManager {
 reloadSourceTableAt(TableDesc.concatResourcePath(tableIdentity));
 }
 
-public void reloadTableCache(String tableIdentity) throws IOException {
-reloadSourceTableExt(tableIdentity);
-reloadSourceTable(tableIdentity);
-}
-
 public DataModelDesc getDataModelDesc(String name) {
 return dataModelDescMap.get(name);
 }
@@ -622,29 +617,4 @@ public class MetadataManager {
 srcTableExdMap.remove(tableIdentity);
 }
 
-public String appendDBName(String table) {
-
-if (table.indexOf(".") > 0)
-return table;
-
-Map map = getAllTablesMap();
-
-int count = 0;
-String result = null;
-for (TableDesc t : map.values()) {
-if (t.getName().equalsIgnoreCase(table)) {
-result = t.getIdentity();
-count++;
-}
-}
-
-if (count == 1)
-return result;
-
-if (count > 1) {
-logger.warn("There are more than 1 table named with '" + table + 
"' in different database; The program couldn't determine, randomly pick '" + 
result + "'");
-}
-return result;
-}
-
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/d5565f77/source-hive/src/main/java/org/apache/kylin/source/hive/SchemaChecker.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/SchemaChecker.java 
b/source-hive/src/main/java/org/apache/kylin/source/hive/SchemaChecker.java
index 3b03551..0c5ccd0 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/SchemaChecker.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/SchemaChecker.java
@@ -134,7 +134,14 @@ public class SchemaChecker {
 }
 }
 
-private List checkAllUsedColumns(CubeInstance cube, TableDesc 
table, Map fieldsMap) {
+/**
+ * check whether all columns used in `cube` has compatible schema in 
current hive schema denoted by `fieldsMap`.
+ * @param cube cube to check, must use `table` in its model
+ * @param table kylin's table metadata
+ * @param fieldsMap current hive schema of `table`
+ * @return true if all columns used in `cube` has compatible schema with 
`fieldsMap`, false otherwise
+ */
+private List checkAllColumnsInCube(CubeInstance cube, TableDesc 
table, Map fieldsMap) {
 Set usedColumns = Sets.newHashSet();
 for (TblColRef col : cube.getAllColumns()) {
 usedColumns.add(col.getColumnDesc());
@@ -152,8 +159,15 @@ public class SchemaChecker {
 return violateColumns;
 }
 
-private boolean checkAllColumns(TableDesc table, List fields) 
{
-if (table.getColumnCount() != fields.size()) {
+/**
+ * check whether all columns in `table` are still in `fields` and have the 
same index as before.
+ *
+ * @param table kylin's table metadata
+ * @param fields current table metadata in hive
+ * @return true if only new columns are appended in hive, false otherwise
+ */
+private boolean checkAllColumnsInTableDesc(TableDesc table, 
List fields) {
+if (table.getColumnCount() > fields.size()) {
 return false;
 }
 
@@ -194,15 +208,16 @@ public class SchemaChecker {
 // if user reloads a fact table used by cube, then all used 

[13/13] kylin git commit: KYLIN-1528 Create a branch for v1.5 with HBase 1.x API

2016-10-30 Thread mahongbin
KYLIN-1528 Create a branch for v1.5 with HBase 1.x API


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

Branch: refs/heads/yang21-hbase1.x
Commit: d1382776e11e52e5a5fc62797aa0aaa9a12b4d72
Parents: 066a38e
Author: shaofengshi 
Authored: Wed Mar 23 17:07:05 2016 +0800
Committer: Hongbin Ma 
Committed: Mon Oct 31 10:21:52 2016 +0800

--
 examples/test_case_data/sandbox/hbase-site.xml  | 19 +---
 .../kylin/provision/BuildCubeWithEngine.java| 13 ++-
 pom.xml | 12 +--
 .../kylin/rest/security/AclHBaseStorage.java|  4 +-
 .../rest/security/MockAclHBaseStorage.java  |  8 +-
 .../apache/kylin/rest/security/MockHTable.java  | 95 
 .../rest/security/RealAclHBaseStorage.java  |  9 +-
 .../apache/kylin/rest/service/AclService.java   | 25 +++---
 .../apache/kylin/rest/service/CubeService.java  | 36 +++-
 .../apache/kylin/rest/service/QueryService.java | 24 +++--
 .../apache/kylin/rest/service/UserService.java  | 17 ++--
 .../kylin/storage/hbase/HBaseConnection.java| 44 -
 .../kylin/storage/hbase/HBaseResourceStore.java | 31 +++
 .../kylin/storage/hbase/HBaseStorage.java   |  3 +-
 .../storage/hbase/cube/SimpleHBaseStore.java| 20 ++---
 .../hbase/cube/v1/CubeSegmentTupleIterator.java | 11 +--
 .../storage/hbase/cube/v1/CubeStorageQuery.java |  6 +-
 .../hbase/cube/v1/RegionScannerAdapter.java | 10 ++-
 .../cube/v1/SerializedHBaseTupleIterator.java   |  4 +-
 .../observer/AggregateRegionObserver.java   |  4 +-
 .../observer/AggregationScanner.java| 14 ++-
 .../observer/ObserverAggregationCache.java  | 10 ++-
 .../coprocessor/observer/ObserverEnabler.java   |  4 +-
 .../hbase/cube/v2/CubeHBaseEndpointRPC.java | 13 +--
 .../storage/hbase/cube/v2/CubeHBaseScanRPC.java |  9 +-
 .../coprocessor/endpoint/CubeVisitService.java  |  4 +-
 .../storage/hbase/steps/CubeHTableUtil.java | 16 ++--
 .../storage/hbase/steps/DeprecatedGCStep.java   | 23 ++---
 .../storage/hbase/steps/HBaseCuboidWriter.java  |  7 +-
 .../hbase/steps/HBaseStreamingOutput.java   |  9 +-
 .../kylin/storage/hbase/steps/MergeGCStep.java  | 23 ++---
 .../storage/hbase/util/CleanHtableCLI.java  | 12 +--
 .../storage/hbase/util/CubeMigrationCLI.java| 36 
 .../hbase/util/CubeMigrationCheckCLI.java   | 17 ++--
 .../hbase/util/DeployCoprocessorCLI.java| 22 ++---
 .../hbase/util/ExtendCubeToHybridCLI.java   |  8 +-
 .../hbase/util/GridTableHBaseBenchmark.java | 34 +++
 .../kylin/storage/hbase/util/HBaseClean.java| 18 ++--
 .../hbase/util/HBaseRegionSizeCalculator.java   | 35 
 .../kylin/storage/hbase/util/HBaseUsage.java|  9 +-
 .../storage/hbase/util/HbaseStreamingInput.java | 30 +++
 .../hbase/util/HtableAlterMetadataCLI.java  |  9 +-
 .../storage/hbase/util/OrphanHBaseCleanJob.java | 19 ++--
 .../kylin/storage/hbase/util/PingHBaseCLI.java  | 15 ++--
 .../kylin/storage/hbase/util/RowCounterCLI.java | 11 +--
 .../storage/hbase/util/StorageCleanupJob.java   | 20 +++--
 .../storage/hbase/util/UpdateHTableHostCLI.java | 17 ++--
 .../observer/AggregateRegionObserverTest.java   | 26 ++
 .../v1/filter/TestFuzzyRowFilterV2EndToEnd.java |  5 +-
 49 files changed, 408 insertions(+), 462 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/d1382776/examples/test_case_data/sandbox/hbase-site.xml
--
diff --git a/examples/test_case_data/sandbox/hbase-site.xml 
b/examples/test_case_data/sandbox/hbase-site.xml
index 46d5345..734908e 100644
--- a/examples/test_case_data/sandbox/hbase-site.xml
+++ b/examples/test_case_data/sandbox/hbase-site.xml
@@ -190,22 +190,5 @@
 zookeeper.znode.parent
 /hbase-unsecure
 
-
-hbase.client.pause
-100
-General client pause value.  Used mostly as value to wait
-before running a retry of a failed get, region lookup, etc.
-See hbase.client.retries.number for description of how we backoff 
from
-this initial pause amount and how this pause works w/ 
retries.
-
-
-hbase.client.retries.number
-5
-Maximum retries.  Used as maximum for all retryable
-operations such as the getting of a cell's value, starting a row 
update,
-etc.  Retry interval is a rough function based on 
hbase.client.pause.  At
-first we retry at this interval but then with backoff, we pretty 
quickly reach
-retrying every ten seconds.  See 

[01/13] kylin git commit: KYLIN-2078 Can't see generated SQL at Web UI [Forced Update!]

2016-10-30 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/yang21-hbase1.x 8948ec7de -> d1382776e (forced update)


KYLIN-2078 Can't see generated SQL at Web UI


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

Branch: refs/heads/yang21-hbase1.x
Commit: f56c3c40a8a80990c4624f5a35f61d0ef3f61f36
Parents: 242a72f
Author: shaofengshi 
Authored: Tue Oct 25 12:57:46 2016 +0800
Committer: shaofengshi 
Committed: Tue Oct 25 13:54:05 2016 +0800

--
 .../java/org/apache/kylin/cube/CubeSegment.java |  1 +
 .../java/org/apache/kylin/cube/ISegment.java| 39 
 .../cube/gridtable/SegmentGTStartAndEnd.java|  2 +-
 .../cube/model/CubeJoinedFlatTableDesc.java |  6 +++
 .../cube/model/CubeJoinedFlatTableEnrich.java   |  6 +++
 .../apache/kylin/gridtable/ScannerWorker.java   |  2 +-
 .../org/apache/kylin/job/JoinedFlatTable.java   | 18 +
 .../metadata/model/IJoinedFlatTableDesc.java|  2 +
 .../apache/kylin/metadata/model/ISegment.java   | 36 ++
 .../kylin/rest/controller/CubeController.java   | 13 +--
 .../hbase/cube/v2/CubeHBaseEndpointRPC.java |  2 +-
 .../storage/hbase/cube/v2/CubeHBaseRPC.java |  2 +-
 .../storage/hbase/cube/v2/CubeHBaseScanRPC.java |  2 +-
 13 files changed, 75 insertions(+), 56 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/f56c3c40/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
--
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java 
b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
index 79397c3..afb0d28 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
@@ -37,6 +37,7 @@ import org.apache.kylin.cube.kv.RowConstants;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.metadata.model.DataModelDesc;
 import org.apache.kylin.metadata.model.IBuildable;
+import org.apache.kylin.metadata.model.ISegment;
 import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.IRealization;

http://git-wip-us.apache.org/repos/asf/kylin/blob/f56c3c40/core-cube/src/main/java/org/apache/kylin/cube/ISegment.java
--
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/ISegment.java 
b/core-cube/src/main/java/org/apache/kylin/cube/ISegment.java
deleted file mode 100644
index 2e1f214..000
--- a/core-cube/src/main/java/org/apache/kylin/cube/ISegment.java
+++ /dev/null
@@ -1,39 +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.
- */
-
-package org.apache.kylin.cube;
-
-import org.apache.kylin.metadata.model.DataModelDesc;
-import org.apache.kylin.metadata.model.SegmentStatusEnum;
-
-public interface ISegment {
-
-public String getName();
-
-public long getDateRangeStart();
-
-public long getDateRangeEnd();
-
-public long getSourceOffsetStart();
-
-public long getSourceOffsetEnd();
-
-public DataModelDesc getModel();
-
-public SegmentStatusEnum getStatus();
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/f56c3c40/core-cube/src/main/java/org/apache/kylin/cube/gridtable/SegmentGTStartAndEnd.java
--
diff --git 
a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/SegmentGTStartAndEnd.java
 
b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/SegmentGTStartAndEnd.java
index 21e01b9..b4a82d4 100644
--- 
a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/SegmentGTStartAndEnd.java
+++ 
b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/SegmentGTStartAndEnd.java
@@ -24,7 +24,7 

[08/13] kylin git commit: KYLIN-2134 dont reuse Object[] returned to calcite

2016-10-30 Thread mahongbin
KYLIN-2134 dont reuse Object[] returned to calcite


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

Branch: refs/heads/yang21-hbase1.x
Commit: 5d1db92bea3bc05f02ca4e411100d58526a51cdf
Parents: 5e3ea3c
Author: Li Yang 
Authored: Thu Oct 27 17:09:22 2016 +0800
Committer: Li Yang 
Committed: Thu Oct 27 17:10:53 2016 +0800

--
 .../kylin/storage/gtrecord/SegmentCubeTupleIterator.java | 4 
 .../org/apache/kylin/query/enumerator/OLAPEnumerator.java| 8 
 2 files changed, 4 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/5d1db92b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SegmentCubeTupleIterator.java
--
diff --git 
a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SegmentCubeTupleIterator.java
 
b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SegmentCubeTupleIterator.java
index 61267ae..00ba247 100644
--- 
a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SegmentCubeTupleIterator.java
+++ 
b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SegmentCubeTupleIterator.java
@@ -37,8 +37,6 @@ import org.apache.kylin.storage.StorageContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Preconditions;
-
 public class SegmentCubeTupleIterator implements ITupleIterator {
 
 private static final Logger logger = 
LoggerFactory.getLogger(SegmentCubeTupleIterator.class);
@@ -98,8 +96,6 @@ public class SegmentCubeTupleIterator implements 
ITupleIterator {
 }
 GTRecord curRecord = gtItr.next();
 
-Preconditions.checkNotNull(cubeTupleConverter);
-
 // translate into tuple
 advMeasureFillers = cubeTupleConverter.translateResult(curRecord, 
tuple);
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/5d1db92b/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java
--
diff --git 
a/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java 
b/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java
index dafbb5f..c7b3c71 100644
--- a/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java
+++ b/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java
@@ -18,6 +18,7 @@
 
 package org.apache.kylin.query.enumerator;
 
+import java.util.Arrays;
 import java.util.Map;
 import java.util.Properties;
 
@@ -77,10 +78,9 @@ public class OLAPEnumerator implements Enumerator {
 }
 
 private Object[] convertCurrentRow(ITuple tuple) {
-// make sure the tuple layout is correct
-//assert 
tuple.getAllFields().equals(olapContext.returnTupleInfo.getAllFields());
-
-current = tuple.getAllValues();
+// give calcite a new array every time, see details in KYLIN-2134
+Object[] values = tuple.getAllValues();
+current = Arrays.copyOf(values, values.length);
 return current;
 }
 



[06/13] kylin git commit: minor, drop kylin.job.step.timeout

2016-10-30 Thread mahongbin
minor, drop kylin.job.step.timeout


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

Branch: refs/heads/yang21-hbase1.x
Commit: 9287117e17bcd5d9f7c7b93bfa5b5d3b2ad7b0c3
Parents: 2d87fae
Author: Li Yang 
Authored: Thu Oct 27 15:11:20 2016 +0800
Committer: Li Yang 
Committed: Thu Oct 27 15:11:20 2016 +0800

--
 build/conf/kylin_job_conf_inmem.xml   | 4 ++--
 .../main/java/org/apache/kylin/common/KylinConfigBase.java| 4 
 .../java/org/apache/kylin/job/engine/JobEngineConfig.java | 7 ---
 3 files changed, 2 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/9287117e/build/conf/kylin_job_conf_inmem.xml
--
diff --git a/build/conf/kylin_job_conf_inmem.xml 
b/build/conf/kylin_job_conf_inmem.xml
index 73b5cc5..d6799d5 100644
--- a/build/conf/kylin_job_conf_inmem.xml
+++ b/build/conf/kylin_job_conf_inmem.xml
@@ -81,8 +81,8 @@
 
 
 mapred.task.timeout
-360
-Set task timeout to 1 hour
+720
+Set task timeout to 2 hour
 
 
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/9287117e/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
--
diff --git 
a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java 
b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index 53074fd..f824d8e 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -407,10 +407,6 @@ abstract public class KylinConfigBase implements 
Serializable {
 return getOptionalStringArray("kylin.job.admin.dls", null);
 }
 
-public long getJobStepTimeout() {
-return Long.parseLong(getOptional("kylin.job.step.timeout", 
String.valueOf(2 * 60 * 60)));
-}
-
 public double getJobCuboidSizeRatio() {
 return Double.parseDouble(getOptional("kylin.job.cuboid.size.ratio", 
"0.25"));
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/9287117e/core-job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
--
diff --git 
a/core-job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java 
b/core-job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
index 8ba1730..1e3eeaa 100644
--- a/core-job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
+++ b/core-job/src/main/java/org/apache/kylin/job/engine/JobEngineConfig.java
@@ -145,13 +145,6 @@ public class JobEngineConfig {
 }
 
 /**
- * @return the jobStepTimeout
- */
-public long getJobStepTimeout() {
-return config.getJobStepTimeout();
-}
-
-/**
  * @return the asyncJobCheckInterval
  */
 public int getAsyncJobCheckInterval() {



[03/13] kylin git commit: KYLIN-2125 add BeelineHiveClient

2016-10-30 Thread mahongbin
KYLIN-2125 add BeelineHiveClient


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

Branch: refs/heads/yang21-hbase1.x
Commit: 6fc1c865ae35a86839e226ae2ca7b3f88661c1dd
Parents: d5565f7
Author: Hongbin Ma 
Authored: Thu Oct 13 15:31:11 2016 +0800
Committer: Hongbin Ma 
Committed: Wed Oct 26 18:43:34 2016 +0800

--
 .../kylin/rest/controller/TableController.java  |   9 +-
 source-hive/pom.xml |   5 +
 .../kylin/source/hive/BeelineHiveClient.java| 214 +++
 .../source/hive/BeelineOptionsProcessor.java|  47 
 .../apache/kylin/source/hive/CLIHiveClient.java | 169 +++
 .../apache/kylin/source/hive/HiveClient.java| 170 ---
 .../kylin/source/hive/HiveClientFactory.java|  33 +++
 .../source/hive/HiveSourceTableLoader.java  |  88 
 .../org/apache/kylin/source/hive/HiveTable.java |  16 +-
 .../apache/kylin/source/hive/HiveTableMeta.java |  71 ++
 .../kylin/source/hive/HiveTableMetaBuilder.java | 102 +
 .../apache/kylin/source/hive/HqlExecutable.java | 107 --
 .../apache/kylin/source/hive/IHiveClient.java   |  36 
 .../hive/BeelineOptionsProcessorTest.java   |  38 
 14 files changed, 769 insertions(+), 336 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/6fc1c865/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
--
diff --git 
a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
 
b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
index eefeba8..b7dd728 100644
--- 
a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
+++ 
b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
@@ -47,7 +47,8 @@ import org.apache.kylin.rest.service.KafkaConfigService;
 import org.apache.kylin.rest.service.ModelService;
 import org.apache.kylin.rest.service.ProjectService;
 import org.apache.kylin.rest.service.StreamingService;
-import org.apache.kylin.source.hive.HiveClient;
+import org.apache.kylin.source.hive.HiveClientFactory;
+import org.apache.kylin.source.hive.IHiveClient;
 import org.apache.kylin.source.kafka.config.KafkaConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -189,7 +190,7 @@ public class TableController extends BasicController {
 String[] dbTableName = HadoopUtil.parseHiveTableName(tableName);
 tableName = dbTableName[0] + "." + dbTableName[1];
 TableDesc desc = 
cubeMgmtService.getMetadataManager().getTableDesc(tableName);
-if(desc == null)
+if (desc == null)
 return false;
 tableType = desc.getSourceType();
 
@@ -312,7 +313,7 @@ public class TableController extends BasicController {
 @RequestMapping(value = "/hive", method = { RequestMethod.GET })
 @ResponseBody
 private static List showHiveDatabases() throws IOException {
-HiveClient hiveClient = new HiveClient();
+IHiveClient hiveClient = HiveClientFactory.getHiveClient();
 List results = null;
 
 try {
@@ -333,7 +334,7 @@ public class TableController extends BasicController {
 @RequestMapping(value = "/hive/{database}", method = { RequestMethod.GET })
 @ResponseBody
 private static List showHiveTables(@PathVariable String database) 
throws IOException {
-HiveClient hiveClient = new HiveClient();
+IHiveClient hiveClient = HiveClientFactory.getHiveClient();
 List results = null;
 
 try {

http://git-wip-us.apache.org/repos/asf/kylin/blob/6fc1c865/source-hive/pom.xml
--
diff --git a/source-hive/pom.xml b/source-hive/pom.xml
index 08019d0..16cb3b4 100644
--- a/source-hive/pom.xml
+++ b/source-hive/pom.xml
@@ -65,6 +65,11 @@
 provided
 
 
+org.apache.hive
+hive-jdbc
+provided
+
+
 org.apache.mrunit
 mrunit
 hadoop2

http://git-wip-us.apache.org/repos/asf/kylin/blob/6fc1c865/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
new file mode 100644
index 000..0fbc39b
--- /dev/null
+++ 

[09/13] kylin git commit: KYLIN-2136 Enhance cubing algorithm selection

2016-10-30 Thread mahongbin
KYLIN-2136 Enhance cubing algorithm selection


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

Branch: refs/heads/yang21-hbase1.x
Commit: 76017797bb65b3f3d5553b90817fe96a5b2d3b32
Parents: 5d1db92
Author: Li Yang 
Authored: Fri Oct 28 11:34:14 2016 +0800
Committer: Li Yang 
Committed: Fri Oct 28 14:40:31 2016 +0800

--
 .../apache/kylin/common/KylinConfigBase.java|  6 +-
 .../kylin/engine/mr/common/CubeStatsReader.java | 13 +++-
 .../kylin/engine/mr/common/CubeStatsWriter.java | 76 
 .../kylin/engine/mr/common/CuboidStatsUtil.java | 73 ---
 .../mr/steps/FactDistinctColumnsReducer.java|  9 ++-
 .../engine/mr/steps/MergeStatisticsStep.java|  4 +-
 .../engine/mr/steps/SaveStatisticsStep.java | 17 +++--
 .../steps/FactDistinctColumnsReducerTest.java   |  4 +-
 .../hbase/steps/HBaseStreamingOutput.java   |  4 +-
 9 files changed, 117 insertions(+), 89 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/76017797/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
--
diff --git 
a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java 
b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index f824d8e..784b2b1 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -420,9 +420,13 @@ abstract public class KylinConfigBase implements 
Serializable {
 }
 
 public double getCubeAlgorithmAutoThreshold() {
-return 
Double.parseDouble(getOptional("kylin.cube.algorithm.auto.threshold", "8"));
+return 
Double.parseDouble(getOptional("kylin.cube.algorithm.auto.threshold", "7"));
 }
 
+public int getCubeAlgorithmAutoMapperLimit() {
+return 
Integer.parseInt(getOptional("kylin.cube.algorithm.auto.mapper.limit", "500"));
+}
+
 @Deprecated
 public int getCubeAggrGroupMaxSize() {
 return Integer.parseInt(getOptional("kylin.cube.aggrgroup.max.size", 
"12"));

http://git-wip-us.apache.org/repos/asf/kylin/blob/76017797/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
--
diff --git 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
index 83e46e3..d31fe48 100644
--- 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
+++ 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/CubeStatsReader.java
@@ -73,7 +73,8 @@ public class CubeStatsReader {
 
 final CubeSegment seg;
 final int samplingPercentage;
-final double mapperOverlapRatioOfFirstBuild; // only makes sense for the 
first build, is meaningless after merge
+final int mapperNumberOfFirstBuild; // becomes meaningless after merge
+final double mapperOverlapRatioOfFirstBuild; // becomes meaningless after 
merge
 final Map cuboidRowEstimatesHLL;
 
 public CubeStatsReader(CubeSegment cubeSegment, KylinConfig kylinConfig) 
throws IOException {
@@ -90,6 +91,7 @@ public class CubeStatsReader {
 reader = new SequenceFile.Reader(hadoopConf, seqInput);
 
 int percentage = 100;
+int mapperNumber = 0;;
 double mapperOverlapRatio = 0;
 Map counterMap = Maps.newHashMap();
 
@@ -100,7 +102,9 @@ public class CubeStatsReader {
 percentage = Bytes.toInt(value.getBytes());
 } else if (key.get() == -1) {
 mapperOverlapRatio = Bytes.toDouble(value.getBytes());
-} else {
+} else if (key.get() == -2) {
+mapperNumber = Bytes.toInt(value.getBytes());
+} else if (key.get() > 0) {
 HyperLogLogPlusCounter hll = new 
HyperLogLogPlusCounter(kylinConfig.getCubeStatsHLLPrecision());
 ByteArray byteArray = new ByteArray(value.getBytes());
 hll.readRegisters(byteArray.asBuffer());
@@ -110,6 +114,7 @@ public class CubeStatsReader {
 
 this.seg = cubeSegment;
 this.samplingPercentage = percentage;
+this.mapperNumberOfFirstBuild = mapperNumber;
 this.mapperOverlapRatioOfFirstBuild = mapperOverlapRatio;
 this.cuboidRowEstimatesHLL = counterMap;
 
@@ 

kylin git commit: KYLIN-2134 dont reuse Object[] returned to calcite

2016-10-27 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/v1.6.0-rc1 1af3d9ee3 -> 8199a90c5


KYLIN-2134 dont reuse Object[] returned to calcite


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

Branch: refs/heads/v1.6.0-rc1
Commit: 8199a90c57df12e8ad4a19737ffa9aa17a69f3eb
Parents: 1af3d9e
Author: Li Yang 
Authored: Thu Oct 27 17:09:22 2016 +0800
Committer: Hongbin Ma 
Committed: Fri Oct 28 09:51:51 2016 +0800

--
 .../kylin/storage/gtrecord/SegmentCubeTupleIterator.java | 4 
 .../org/apache/kylin/query/enumerator/OLAPEnumerator.java| 8 
 2 files changed, 4 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/8199a90c/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SegmentCubeTupleIterator.java
--
diff --git 
a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SegmentCubeTupleIterator.java
 
b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SegmentCubeTupleIterator.java
index 61267ae..00ba247 100644
--- 
a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SegmentCubeTupleIterator.java
+++ 
b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SegmentCubeTupleIterator.java
@@ -37,8 +37,6 @@ import org.apache.kylin.storage.StorageContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.base.Preconditions;
-
 public class SegmentCubeTupleIterator implements ITupleIterator {
 
 private static final Logger logger = 
LoggerFactory.getLogger(SegmentCubeTupleIterator.class);
@@ -98,8 +96,6 @@ public class SegmentCubeTupleIterator implements 
ITupleIterator {
 }
 GTRecord curRecord = gtItr.next();
 
-Preconditions.checkNotNull(cubeTupleConverter);
-
 // translate into tuple
 advMeasureFillers = cubeTupleConverter.translateResult(curRecord, 
tuple);
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8199a90c/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java
--
diff --git 
a/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java 
b/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java
index dafbb5f..c7b3c71 100644
--- a/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java
+++ b/query/src/main/java/org/apache/kylin/query/enumerator/OLAPEnumerator.java
@@ -18,6 +18,7 @@
 
 package org.apache.kylin.query.enumerator;
 
+import java.util.Arrays;
 import java.util.Map;
 import java.util.Properties;
 
@@ -77,10 +78,9 @@ public class OLAPEnumerator implements Enumerator {
 }
 
 private Object[] convertCurrentRow(ITuple tuple) {
-// make sure the tuple layout is correct
-//assert 
tuple.getAllFields().equals(olapContext.returnTupleInfo.getAllFields());
-
-current = tuple.getAllValues();
+// give calcite a new array every time, see details in KYLIN-2134
+Object[] values = tuple.getAllValues();
+current = Arrays.copyOf(values, values.length);
 return current;
 }
 



kylin git commit: KYLIN-2134 Kylin will treat empty string as NULL by mistake

2016-10-27 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/1.5.4.1-beeline 8f4ed7279 -> 3fb0c16c2


KYLIN-2134 Kylin will treat empty string as NULL by mistake


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

Branch: refs/heads/1.5.4.1-beeline
Commit: 3fb0c16c263d8a7edc443d919200f0c244c517fb
Parents: 8f4ed72
Author: Hongbin Ma 
Authored: Thu Oct 27 15:22:54 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Oct 27 15:25:31 2016 +0800

--
 .../src/main/java/org/apache/kylin/metadata/tuple/Tuple.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/3fb0c16c/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
--
diff --git 
a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java 
b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
index aaf9aa9..9074403 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
@@ -178,7 +178,7 @@ public class Tuple implements ITuple {
 if (strValue == null)
 return null;
 
-if ((strValue.equals("") || strValue.equals("\\N")) && 
!dataTypeName.equals("string"))
+if ((strValue.equals("") || strValue.equals("\\N")) && 
!dataTypeName.equals("string") && !dataTypeName.startsWith("varchar"))
 return null;
 
 // TODO use data type enum instead of string comparison



kylin git commit: KYLIN-2134 Kylin will treat empty string as NULL by mistake

2016-10-27 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/v1.6.0-rc1 cd2a06a5d -> 1af3d9ee3


KYLIN-2134 Kylin will treat empty string as NULL by mistake


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

Branch: refs/heads/v1.6.0-rc1
Commit: 1af3d9ee31f3edd64eea336c65b7619b078efdbf
Parents: cd2a06a
Author: Hongbin Ma 
Authored: Thu Oct 27 15:22:54 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Oct 27 15:25:48 2016 +0800

--
 .../src/main/java/org/apache/kylin/metadata/tuple/Tuple.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/1af3d9ee/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
--
diff --git 
a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java 
b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
index aaf9aa9..9074403 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
@@ -178,7 +178,7 @@ public class Tuple implements ITuple {
 if (strValue == null)
 return null;
 
-if ((strValue.equals("") || strValue.equals("\\N")) && 
!dataTypeName.equals("string"))
+if ((strValue.equals("") || strValue.equals("\\N")) && 
!dataTypeName.equals("string") && !dataTypeName.startsWith("varchar"))
 return null;
 
 // TODO use data type enum instead of string comparison



kylin git commit: KYLIN-2134 Kylin will treat empty string as NULL by mistake

2016-10-27 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/yang21 9287117e1 -> 5e3ea3c13


KYLIN-2134 Kylin will treat empty string as NULL by mistake


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

Branch: refs/heads/yang21
Commit: 5e3ea3c13c52513bcfa6781674424276b93bb51d
Parents: 9287117
Author: Hongbin Ma 
Authored: Thu Oct 27 15:22:54 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Oct 27 15:26:07 2016 +0800

--
 .../src/main/java/org/apache/kylin/metadata/tuple/Tuple.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/5e3ea3c1/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
--
diff --git 
a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java 
b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
index aaf9aa9..9074403 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
@@ -178,7 +178,7 @@ public class Tuple implements ITuple {
 if (strValue == null)
 return null;
 
-if ((strValue.equals("") || strValue.equals("\\N")) && 
!dataTypeName.equals("string"))
+if ((strValue.equals("") || strValue.equals("\\N")) && 
!dataTypeName.equals("string") && !dataTypeName.startsWith("varchar"))
 return null;
 
 // TODO use data type enum instead of string comparison



kylin git commit: KYLIN-2134 Kylin will treat empty string as NULL by mistake

2016-10-27 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/master 468fe9ffc -> 49260dd0c


KYLIN-2134 Kylin will treat empty string as NULL by mistake


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

Branch: refs/heads/master
Commit: 49260dd0cb1c129d40b51c359197402441b87256
Parents: 468fe9f
Author: Hongbin Ma 
Authored: Thu Oct 27 15:22:54 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Oct 27 15:22:57 2016 +0800

--
 .../src/main/java/org/apache/kylin/metadata/tuple/Tuple.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/49260dd0/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
--
diff --git 
a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java 
b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
index aaf9aa9..9074403 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/tuple/Tuple.java
@@ -178,7 +178,7 @@ public class Tuple implements ITuple {
 if (strValue == null)
 return null;
 
-if ((strValue.equals("") || strValue.equals("\\N")) && 
!dataTypeName.equals("string"))
+if ((strValue.equals("") || strValue.equals("\\N")) && 
!dataTypeName.equals("string") && !dataTypeName.startsWith("varchar"))
 return null;
 
 // TODO use data type enum instead of string comparison



[3/3] kylin git commit: port KYLIN-2012 to new interface introduced in KYLIN-2125

2016-10-26 Thread mahongbin
port KYLIN-2012 to new interface introduced in KYLIN-2125


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

Branch: refs/heads/yang21
Commit: b502a17414468652b75f84994fb371a767784a47
Parents: 6fc1c86
Author: Hongbin Ma 
Authored: Wed Oct 26 14:04:56 2016 +0800
Committer: Hongbin Ma 
Committed: Wed Oct 26 18:49:34 2016 +0800

--
 .../java/org/apache/kylin/job/DeployUtil.java   |  5 ++-
 .../kylin/source/hive/BeelineHiveClient.java|  6 +--
 .../source/hive/HiveSourceTableLoader.java  | 32 +++
 .../apache/kylin/source/hive/SchemaChecker.java | 41 
 4 files changed, 38 insertions(+), 46 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/b502a174/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
--
diff --git a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java 
b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
index 8c64f91..0734f4f 100644
--- a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -38,8 +38,9 @@ import org.apache.kylin.metadata.MetadataManager;
 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.hive.HiveClient;
+import org.apache.kylin.source.hive.HiveClientFactory;
 import org.apache.kylin.source.hive.HiveCmdBuilder;
+import org.apache.kylin.source.hive.IHiveClient;
 import org.apache.kylin.source.kafka.TimedJsonStreamParser;
 import org.apache.maven.model.Model;
 import org.apache.maven.model.io.xpp3.MavenXpp3Reader;
@@ -205,7 +206,7 @@ public class DeployUtil {
 String tableFileDir = temp.getParent();
 temp.delete();
 
-HiveClient hiveClient = new HiveClient();
+IHiveClient hiveClient = HiveClientFactory.getHiveClient();
 // create hive tables
 hiveClient.executeHQL("CREATE DATABASE IF NOT EXISTS EDW");
 
hiveClient.executeHQL(generateCreateTableHql(metaMgr.getTableDesc(TABLE_CAL_DT.toUpperCase(;

http://git-wip-us.apache.org/repos/asf/kylin/blob/b502a174/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
index 0fbc39b..b027f2e 100644
--- 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
+++ 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
@@ -205,9 +205,9 @@ public class BeelineHiveClient implements IHiveClient {
 
 public static void main(String[] args) throws SQLException {
 
-//BeelineHiveClient loader = new BeelineHiveClient("-n root --hiveconf 
hive.security.authorization.sqlstd.confwhitelist.append='mapreduce.job.*|dfs.*' 
-u 'jdbc:hive2://sandbox:1'");
-BeelineHiveClient loader = new 
BeelineHiveClient(StringUtils.join(args, " "));
-HiveTableMeta hiveTableMeta = loader.getHiveTableMeta("default", 
"events");
+BeelineHiveClient loader = new BeelineHiveClient("-n root --hiveconf 
hive.security.authorization.sqlstd.confwhitelist.append='mapreduce.job.*|dfs.*' 
-u 'jdbc:hive2://sandbox:1'");
+//BeelineHiveClient loader = new 
BeelineHiveClient(StringUtils.join(args, " "));
+HiveTableMeta hiveTableMeta = loader.getHiveTableMeta("default", 
"test_kylin_fact_part");
 System.out.println(hiveTableMeta);
 loader.close();
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/b502a174/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
 
b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
index 346d278..1335ec3 100644
--- 
a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
+++ 
b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
@@ -26,6 +26,7 @@ import java.util.Set;
 import java.util.UUID;
 
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.metadata.MetadataConstants;
 import 

[2/3] kylin git commit: KYLIN-2125 add BeelineHiveClient

2016-10-26 Thread mahongbin
KYLIN-2125 add BeelineHiveClient


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

Branch: refs/heads/yang21
Commit: 6fc1c865ae35a86839e226ae2ca7b3f88661c1dd
Parents: d5565f7
Author: Hongbin Ma 
Authored: Thu Oct 13 15:31:11 2016 +0800
Committer: Hongbin Ma 
Committed: Wed Oct 26 18:43:34 2016 +0800

--
 .../kylin/rest/controller/TableController.java  |   9 +-
 source-hive/pom.xml |   5 +
 .../kylin/source/hive/BeelineHiveClient.java| 214 +++
 .../source/hive/BeelineOptionsProcessor.java|  47 
 .../apache/kylin/source/hive/CLIHiveClient.java | 169 +++
 .../apache/kylin/source/hive/HiveClient.java| 170 ---
 .../kylin/source/hive/HiveClientFactory.java|  33 +++
 .../source/hive/HiveSourceTableLoader.java  |  88 
 .../org/apache/kylin/source/hive/HiveTable.java |  16 +-
 .../apache/kylin/source/hive/HiveTableMeta.java |  71 ++
 .../kylin/source/hive/HiveTableMetaBuilder.java | 102 +
 .../apache/kylin/source/hive/HqlExecutable.java | 107 --
 .../apache/kylin/source/hive/IHiveClient.java   |  36 
 .../hive/BeelineOptionsProcessorTest.java   |  38 
 14 files changed, 769 insertions(+), 336 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/6fc1c865/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
--
diff --git 
a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
 
b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
index eefeba8..b7dd728 100644
--- 
a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
+++ 
b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
@@ -47,7 +47,8 @@ import org.apache.kylin.rest.service.KafkaConfigService;
 import org.apache.kylin.rest.service.ModelService;
 import org.apache.kylin.rest.service.ProjectService;
 import org.apache.kylin.rest.service.StreamingService;
-import org.apache.kylin.source.hive.HiveClient;
+import org.apache.kylin.source.hive.HiveClientFactory;
+import org.apache.kylin.source.hive.IHiveClient;
 import org.apache.kylin.source.kafka.config.KafkaConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -189,7 +190,7 @@ public class TableController extends BasicController {
 String[] dbTableName = HadoopUtil.parseHiveTableName(tableName);
 tableName = dbTableName[0] + "." + dbTableName[1];
 TableDesc desc = 
cubeMgmtService.getMetadataManager().getTableDesc(tableName);
-if(desc == null)
+if (desc == null)
 return false;
 tableType = desc.getSourceType();
 
@@ -312,7 +313,7 @@ public class TableController extends BasicController {
 @RequestMapping(value = "/hive", method = { RequestMethod.GET })
 @ResponseBody
 private static List showHiveDatabases() throws IOException {
-HiveClient hiveClient = new HiveClient();
+IHiveClient hiveClient = HiveClientFactory.getHiveClient();
 List results = null;
 
 try {
@@ -333,7 +334,7 @@ public class TableController extends BasicController {
 @RequestMapping(value = "/hive/{database}", method = { RequestMethod.GET })
 @ResponseBody
 private static List showHiveTables(@PathVariable String database) 
throws IOException {
-HiveClient hiveClient = new HiveClient();
+IHiveClient hiveClient = HiveClientFactory.getHiveClient();
 List results = null;
 
 try {

http://git-wip-us.apache.org/repos/asf/kylin/blob/6fc1c865/source-hive/pom.xml
--
diff --git a/source-hive/pom.xml b/source-hive/pom.xml
index 08019d0..16cb3b4 100644
--- a/source-hive/pom.xml
+++ b/source-hive/pom.xml
@@ -65,6 +65,11 @@
 provided
 
 
+org.apache.hive
+hive-jdbc
+provided
+
+
 org.apache.mrunit
 mrunit
 hadoop2

http://git-wip-us.apache.org/repos/asf/kylin/blob/6fc1c865/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
new file mode 100644
index 000..0fbc39b
--- /dev/null
+++ 

[3/3] kylin git commit: KYLIN-2125 add BeelineHiveClient

2016-10-26 Thread mahongbin
KYLIN-2125 add BeelineHiveClient


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

Branch: refs/heads/v1.6.0-rc1
Commit: 9ed775c35426fea47c9c63a37677d73cddb70e9c
Parents: e094b76
Author: Hongbin Ma 
Authored: Thu Oct 13 15:31:11 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Oct 27 08:34:11 2016 +0800

--
 .../kylin/rest/controller/TableController.java  |   9 +-
 source-hive/pom.xml |   5 +
 .../kylin/source/hive/BeelineHiveClient.java| 214 +++
 .../source/hive/BeelineOptionsProcessor.java|  47 
 .../apache/kylin/source/hive/CLIHiveClient.java | 169 +++
 .../apache/kylin/source/hive/HiveClient.java| 170 ---
 .../kylin/source/hive/HiveClientFactory.java|  33 +++
 .../source/hive/HiveSourceTableLoader.java  |  89 
 .../org/apache/kylin/source/hive/HiveTable.java |  16 +-
 .../apache/kylin/source/hive/HiveTableMeta.java |  71 ++
 .../kylin/source/hive/HiveTableMetaBuilder.java | 102 +
 .../apache/kylin/source/hive/HqlExecutable.java | 107 --
 .../apache/kylin/source/hive/IHiveClient.java   |  36 
 .../hive/BeelineOptionsProcessorTest.java   |  38 
 14 files changed, 769 insertions(+), 337 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/9ed775c3/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
--
diff --git 
a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
 
b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
index d9050c1..47ff3fe 100644
--- 
a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
+++ 
b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
@@ -47,7 +47,8 @@ import org.apache.kylin.rest.service.KafkaConfigService;
 import org.apache.kylin.rest.service.ModelService;
 import org.apache.kylin.rest.service.ProjectService;
 import org.apache.kylin.rest.service.StreamingService;
-import org.apache.kylin.source.hive.HiveClient;
+import org.apache.kylin.source.hive.HiveClientFactory;
+import org.apache.kylin.source.hive.IHiveClient;
 import org.apache.kylin.source.kafka.config.KafkaConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -189,7 +190,7 @@ public class TableController extends BasicController {
 String[] dbTableName = HadoopUtil.parseHiveTableName(tableName);
 tableName = dbTableName[0] + "." + dbTableName[1];
 TableDesc desc = 
cubeMgmtService.getMetadataManager().getTableDesc(tableName);
-if(desc == null)
+if (desc == null)
 return false;
 tableType = desc.getSourceType();
 
@@ -315,7 +316,7 @@ public class TableController extends BasicController {
 @RequestMapping(value = "/hive", method = { RequestMethod.GET })
 @ResponseBody
 private static List showHiveDatabases() throws IOException {
-HiveClient hiveClient = new HiveClient();
+IHiveClient hiveClient = HiveClientFactory.getHiveClient();
 List results = null;
 
 try {
@@ -336,7 +337,7 @@ public class TableController extends BasicController {
 @RequestMapping(value = "/hive/{database}", method = { RequestMethod.GET })
 @ResponseBody
 private static List showHiveTables(@PathVariable String database) 
throws IOException {
-HiveClient hiveClient = new HiveClient();
+IHiveClient hiveClient = HiveClientFactory.getHiveClient();
 List results = null;
 
 try {

http://git-wip-us.apache.org/repos/asf/kylin/blob/9ed775c3/source-hive/pom.xml
--
diff --git a/source-hive/pom.xml b/source-hive/pom.xml
index 08019d0..16cb3b4 100644
--- a/source-hive/pom.xml
+++ b/source-hive/pom.xml
@@ -65,6 +65,11 @@
 provided
 
 
+org.apache.hive
+hive-jdbc
+provided
+
+
 org.apache.mrunit
 mrunit
 hadoop2

http://git-wip-us.apache.org/repos/asf/kylin/blob/9ed775c3/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
new file mode 100644
index 000..0fbc39b
--- /dev/null
+++ 

[2/3] kylin git commit: port KYLIN-2068 to BeelineHiveClient(KYLIN-2125)

2016-10-26 Thread mahongbin
port KYLIN-2068 to BeelineHiveClient(KYLIN-2125)


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

Branch: refs/heads/v1.6.0-rc1
Commit: 553d7c56a0af49cb1c003f94543b4e600cd6f66e
Parents: 9ed775c
Author: Hongbin Ma 
Authored: Wed Oct 26 13:37:03 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Oct 27 08:34:11 2016 +0800

--
 .../org/apache/kylin/source/hive/BeelineHiveClient.java   | 10 +-
 .../java/org/apache/kylin/source/hive/CLIHiveClient.java  |  4 ++--
 .../apache/kylin/source/hive/HiveSourceTableLoader.java   |  1 +
 .../java/org/apache/kylin/source/hive/HiveTableMeta.java  |  6 --
 4 files changed, 12 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/553d7c56/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
index 0fbc39b..c8d56a5 100644
--- 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
+++ 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
@@ -115,7 +115,7 @@ public class BeelineHiveClient implements IHiveClient {
 
 List allColumns = 
Lists.newArrayList();
 while (columns.next()) {
-allColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(columns.getString(4), columns.getString(6)));
+allColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(columns.getString(4), columns.getString(6), 
columns.getString(12)));
 }
 builder.setAllColumns(allColumns);
 
@@ -139,7 +139,7 @@ public class BeelineHiveClient implements IHiveClient {
 if ("".equals(resultSet.getString(1).trim())) {
 break;
 }
-partitionColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(resultSet.getString(1).trim(), 
resultSet.getString(2).trim()));
+partitionColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(resultSet.getString(1).trim(), 
resultSet.getString(2).trim(), resultSet.getString(3).trim()));
 }
 builder.setPartitionColumns(partitionColumns);
 }
@@ -205,9 +205,9 @@ public class BeelineHiveClient implements IHiveClient {
 
 public static void main(String[] args) throws SQLException {
 
-//BeelineHiveClient loader = new BeelineHiveClient("-n root --hiveconf 
hive.security.authorization.sqlstd.confwhitelist.append='mapreduce.job.*|dfs.*' 
-u 'jdbc:hive2://sandbox:1'");
-BeelineHiveClient loader = new 
BeelineHiveClient(StringUtils.join(args, " "));
-HiveTableMeta hiveTableMeta = loader.getHiveTableMeta("default", 
"events");
+BeelineHiveClient loader = new BeelineHiveClient("-n root --hiveconf 
hive.security.authorization.sqlstd.confwhitelist.append='mapreduce.job.*|dfs.*' 
-u 'jdbc:hive2://sandbox:1'");
+//BeelineHiveClient loader = new 
BeelineHiveClient(StringUtils.join(args, " "));
+HiveTableMeta hiveTableMeta = loader.getHiveTableMeta("default", 
"test001");
 System.out.println(hiveTableMeta);
 loader.close();
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/553d7c56/source-hive/src/main/java/org/apache/kylin/source/hive/CLIHiveClient.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/CLIHiveClient.java 
b/source-hive/src/main/java/org/apache/kylin/source/hive/CLIHiveClient.java
index ea74470..60cf47a 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/CLIHiveClient.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/CLIHiveClient.java
@@ -89,11 +89,11 @@ public class CLIHiveClient implements IHiveClient {
 List allColumns = 
Lists.newArrayList();
 List partitionColumns = 
Lists.newArrayList();
 for (FieldSchema fieldSchema : allFields) {
-allColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(fieldSchema.getName(), 
fieldSchema.getType()));
+allColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(fieldSchema.getName(), fieldSchema.getType(), 
fieldSchema.getComment()));
 }
 if (partitionFields != null && partitionFields.size() > 0) {
 for (FieldSchema fieldSchema : partitionFields) {
-partitionColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(fieldSchema.getName(), 

[1/3] kylin git commit: port KYLIN-2012 to new interface introduced in KYLIN-2125

2016-10-26 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/v1.6.0-rc1 e094b7687 -> cd2a06a5d


port KYLIN-2012 to new interface introduced in KYLIN-2125


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

Branch: refs/heads/v1.6.0-rc1
Commit: cd2a06a5d373bdd5cfa90e78649d42e891711c43
Parents: 553d7c5
Author: Hongbin Ma 
Authored: Wed Oct 26 14:04:56 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Oct 27 08:34:11 2016 +0800

--
 .../java/org/apache/kylin/job/DeployUtil.java   |  5 ++-
 .../kylin/source/hive/BeelineHiveClient.java|  2 +-
 .../source/hive/HiveSourceTableLoader.java  | 32 +++
 .../apache/kylin/source/hive/SchemaChecker.java | 41 
 4 files changed, 36 insertions(+), 44 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/cd2a06a5/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
--
diff --git a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java 
b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
index 4a24ad2..54feb24 100644
--- a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -45,8 +45,9 @@ import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TableRef;
 import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.source.hive.HiveClient;
+import org.apache.kylin.source.hive.HiveClientFactory;
 import org.apache.kylin.source.hive.HiveCmdBuilder;
+import org.apache.kylin.source.hive.IHiveClient;
 import org.apache.kylin.source.kafka.TimedJsonStreamParser;
 import org.apache.maven.model.Model;
 import org.apache.maven.model.io.xpp3.MavenXpp3Reader;
@@ -234,7 +235,7 @@ public class DeployUtil {
 String tableFileDir = temp.getParent();
 temp.delete();
 
-HiveClient hiveClient = new HiveClient();
+IHiveClient hiveClient = HiveClientFactory.getHiveClient();
 // create hive tables
 hiveClient.executeHQL("CREATE DATABASE IF NOT EXISTS EDW");
 
hiveClient.executeHQL(generateCreateTableHql(metaMgr.getTableDesc(TABLE_CAL_DT.toUpperCase(;

http://git-wip-us.apache.org/repos/asf/kylin/blob/cd2a06a5/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
index c8d56a5..a84aeb1 100644
--- 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
+++ 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
@@ -207,7 +207,7 @@ public class BeelineHiveClient implements IHiveClient {
 
 BeelineHiveClient loader = new BeelineHiveClient("-n root --hiveconf 
hive.security.authorization.sqlstd.confwhitelist.append='mapreduce.job.*|dfs.*' 
-u 'jdbc:hive2://sandbox:1'");
 //BeelineHiveClient loader = new 
BeelineHiveClient(StringUtils.join(args, " "));
-HiveTableMeta hiveTableMeta = loader.getHiveTableMeta("default", 
"test001");
+HiveTableMeta hiveTableMeta = loader.getHiveTableMeta("default", 
"test_kylin_fact_part");
 System.out.println(hiveTableMeta);
 loader.close();
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/cd2a06a5/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
 
b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
index 388e72b..401e720 100644
--- 
a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
+++ 
b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
@@ -26,6 +26,7 @@ import java.util.Set;
 import java.util.UUID;
 
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.metadata.MetadataConstants;
 import org.apache.kylin.metadata.MetadataManager;
@@ -34,8 +35,10 @@ import org.apache.kylin.metadata.model.TableDesc;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.LinkedHashMultimap;
 import com.google.common.collect.Lists;
 import 

[2/7] kylin git commit: KYLIN-2030 enhancement

2016-10-26 Thread mahongbin
KYLIN-2030 enhancement


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

Branch: refs/heads/master
Commit: 46284bdca211d59f4a3274f5579dc8fe8a48cc6e
Parents: 1429da5
Author: Hongbin Ma 
Authored: Sat Oct 8 16:14:26 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Oct 27 08:30:13 2016 +0800

--
 .../kylin/cube/CubeCapabilityChecker.java   | 43 +++-
 .../org/apache/kylin/cube/model/CubeDesc.java   |  6 +--
 .../kylin/metadata/model/FunctionDesc.java  |  1 +
 .../test/resources/query/sql_lookup/query06.sql |  1 +
 .../test/resources/query/sql_lookup/query07.sql |  1 +
 5 files changed, 39 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/46284bdc/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
--
diff --git 
a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java 
b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
index c8460f0..ee21b1c 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
@@ -25,18 +25,22 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.measure.MeasureType;
 import org.apache.kylin.measure.basic.BasicMeasureType;
+import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.filter.UDF.MassInTupleFilter;
+import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.IStorageAware;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.ParameterDesc;
+import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.CapabilityResult;
-import 
org.apache.kylin.metadata.realization.CapabilityResult.CapabilityInfluence;
 import org.apache.kylin.metadata.realization.SQLDigest;
+import 
org.apache.kylin.metadata.realization.CapabilityResult.CapabilityInfluence;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -66,20 +70,39 @@ public class CubeCapabilityChecker {
 Collection unmatchedAggregations = 
unmatchedAggregations(aggrFunctions, cube);
 
 // try custom measure types
+// in RAW query, unmatchedDimensions and unmatchedAggregations will 
null, so can't chose RAW cube well!
+//if (!unmatchedDimensions.isEmpty() || 
!unmatchedAggregations.isEmpty()) {
 tryCustomMeasureTypes(unmatchedDimensions, unmatchedAggregations, 
digest, cube, result);
+//}
 
-// try dimension-as-measure
-if (!unmatchedAggregations.isEmpty()) {
-if (cube.getDescriptor().getFactTable().equals(digest.factTable)) {
+//more tricks
+if (cube.getDescriptor().getFactTable().equals(digest.factTable)) {
+//for query-on-facttable
+//1. dimension as measure
+
+if (!unmatchedAggregations.isEmpty()) {
 tryDimensionAsMeasures(unmatchedAggregations, digest, cube, 
result, cube.getDescriptor().listDimensionColumnsIncludingDerived());
-} else {
-//deal with query on lookup table, like 
https://issues.apache.org/jira/browse/KYLIN-2030
-if 
(cube.getSegments().get(0).getSnapshots().containsKey(digest.factTable)) {
-Set dimCols = 
Sets.newHashSet(cube.getDataModelDesc().findFirstTable(digest.factTable).getColumns());
+}
+} else {
+//for non query-on-facttable 
+if 
(cube.getSegments().get(0).getSnapshots().containsKey(digest.factTable)) {
+TableDesc tableDesc = 
MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getTableDesc(digest.factTable);
+Set dimCols = Sets.newHashSet();
+for (ColumnDesc columnDesc : tableDesc.getColumns()) {
+dimCols.add(columnDesc.getRef());
+}
+
+//1. dimension as measure, like max(cal_dt) or count( distinct 
col) from lookup
+if (!unmatchedAggregations.isEmpty()) {
 tryDimensionAsMeasures(unmatchedAggregations, digest, 
cube, result, dimCols);
-} else {
-logger.info("Skip tryDimensionAsMeasures 

[7/7] kylin git commit: improve integer type upgrade logic

2016-10-26 Thread mahongbin
improve integer type upgrade logic


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

Branch: refs/heads/master
Commit: 1429da5f650852402d1494785368e9175f13c379
Parents: 615e21d
Author: Hongbin Ma 
Authored: Fri Sep 30 18:31:04 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Oct 27 08:30:13 2016 +0800

--
 .../filter/EvaluatableFunctionTupleFilter.java  | 151 ---
 .../metadata/filter/TupleFilterSerializer.java  |  20 ++-
 .../apache/kylin/metadata/model/ColumnDesc.java |  20 ++-
 .../apache/kylin/metadata/tuple/TupleInfo.java  |   4 +-
 .../apache/kylin/query/schema/OLAPTable.java|  31 +++-
 5 files changed, 63 insertions(+), 163 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/1429da5f/core-metadata/src/main/java/org/apache/kylin/metadata/filter/EvaluatableFunctionTupleFilter.java
--
diff --git 
a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/EvaluatableFunctionTupleFilter.java
 
b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/EvaluatableFunctionTupleFilter.java
deleted file mode 100644
index ff24172..000
--- 
a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/EvaluatableFunctionTupleFilter.java
+++ /dev/null
@@ -1,151 +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.
- */
-
-package org.apache.kylin.metadata.filter;
-
-import java.lang.reflect.InvocationTargetException;
-import java.nio.ByteBuffer;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.datatype.DataType;
-import org.apache.kylin.metadata.datatype.StringSerializer;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.metadata.tuple.IEvaluatableTuple;
-
-import com.google.common.collect.Lists;
-
-public class EvaluatableFunctionTupleFilter extends BuiltInFunctionTupleFilter 
{
-
-private boolean constantsInitted = false;
-
-//about non-like
-private List values;
-private Object tupleValue;
-
-public EvaluatableFunctionTupleFilter(String name) {
-super(name, FilterOperatorEnum.EVAL_FUNC);
-values = Lists.newArrayListWithCapacity(1);
-values.add(null);
-}
-
-@Override
-public boolean evaluate(IEvaluatableTuple tuple, IFilterCodeSystem cs) {
-
-// extract tuple value
-Object tupleValue = null;
-for (TupleFilter filter : this.children) {
-if (!isConstant(filter)) {
-filter.evaluate(tuple, cs);
-tupleValue = filter.getValues().iterator().next();
-}
-}
-
-TblColRef tblColRef = this.getColumn();
-DataType strDataType = DataType.getType("string");
-if (tblColRef.getType() != strDataType) {
-throw new IllegalStateException("Only String type is allow in 
BuiltInFunction");
-}
-ByteArray valueByteArray = (ByteArray) tupleValue;
-StringSerializer serializer = new StringSerializer(strDataType);
-String value = 
serializer.deserialize(ByteBuffer.wrap(valueByteArray.array(), 
valueByteArray.offset(), valueByteArray.length()));
-
-try {
-if (isLikeFunction()) {
-return (Boolean) invokeFunction(value);
-} else {
-this.tupleValue = invokeFunction(value);
-//convert back to ByteArray format because the outer 
EvaluatableFunctionTupleFilter assumes input as ByteArray
-ByteBuffer buffer = 
ByteBuffer.allocate(valueByteArray.length() * 2);
-serializer.serialize((String) this.tupleValue, buffer);
-this.tupleValue = new ByteArray(buffer.array(), 0, 
buffer.position());
-
-

[5/7] kylin git commit: port KYLIN-2012 to new interface introduced in KYLIN-2125

2016-10-26 Thread mahongbin
port KYLIN-2012 to new interface introduced in KYLIN-2125


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

Branch: refs/heads/master
Commit: 615e21d7fb0dae651ab643949ef5078bef7b12ea
Parents: ca6837d
Author: Hongbin Ma 
Authored: Wed Oct 26 14:04:56 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Oct 27 08:30:13 2016 +0800

--
 .../java/org/apache/kylin/job/DeployUtil.java   |  5 ++-
 .../kylin/source/hive/BeelineHiveClient.java|  2 +-
 .../source/hive/HiveSourceTableLoader.java  | 32 +++
 .../apache/kylin/source/hive/SchemaChecker.java | 41 
 4 files changed, 36 insertions(+), 44 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/615e21d7/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
--
diff --git a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java 
b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
index 4a24ad2..54feb24 100644
--- a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -45,8 +45,9 @@ import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TableRef;
 import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.source.hive.HiveClient;
+import org.apache.kylin.source.hive.HiveClientFactory;
 import org.apache.kylin.source.hive.HiveCmdBuilder;
+import org.apache.kylin.source.hive.IHiveClient;
 import org.apache.kylin.source.kafka.TimedJsonStreamParser;
 import org.apache.maven.model.Model;
 import org.apache.maven.model.io.xpp3.MavenXpp3Reader;
@@ -234,7 +235,7 @@ public class DeployUtil {
 String tableFileDir = temp.getParent();
 temp.delete();
 
-HiveClient hiveClient = new HiveClient();
+IHiveClient hiveClient = HiveClientFactory.getHiveClient();
 // create hive tables
 hiveClient.executeHQL("CREATE DATABASE IF NOT EXISTS EDW");
 
hiveClient.executeHQL(generateCreateTableHql(metaMgr.getTableDesc(TABLE_CAL_DT.toUpperCase(;

http://git-wip-us.apache.org/repos/asf/kylin/blob/615e21d7/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
index c8d56a5..a84aeb1 100644
--- 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
+++ 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
@@ -207,7 +207,7 @@ public class BeelineHiveClient implements IHiveClient {
 
 BeelineHiveClient loader = new BeelineHiveClient("-n root --hiveconf 
hive.security.authorization.sqlstd.confwhitelist.append='mapreduce.job.*|dfs.*' 
-u 'jdbc:hive2://sandbox:1'");
 //BeelineHiveClient loader = new 
BeelineHiveClient(StringUtils.join(args, " "));
-HiveTableMeta hiveTableMeta = loader.getHiveTableMeta("default", 
"test001");
+HiveTableMeta hiveTableMeta = loader.getHiveTableMeta("default", 
"test_kylin_fact_part");
 System.out.println(hiveTableMeta);
 loader.close();
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/615e21d7/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
 
b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
index 388e72b..401e720 100644
--- 
a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
+++ 
b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
@@ -26,6 +26,7 @@ import java.util.Set;
 import java.util.UUID;
 
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.metadata.MetadataConstants;
 import org.apache.kylin.metadata.MetadataManager;
@@ -34,8 +35,10 @@ import org.apache.kylin.metadata.model.TableDesc;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.LinkedHashMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import com.google.common.collect.SetMultimap;
 import 

[1/7] kylin git commit: KYLIN-2030 fix with KYLIN-1971

2016-10-26 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/master ea3d02ff1 -> c804dc8d7


KYLIN-2030 fix with KYLIN-1971


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

Branch: refs/heads/master
Commit: c804dc8d77bb0402d6cd391e582a47f323edbb21
Parents: 324e423
Author: Hongbin Ma 
Authored: Wed Oct 26 14:41:06 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Oct 27 08:30:13 2016 +0800

--
 .../org/apache/kylin/cube/CubeCapabilityChecker.java   | 13 +++--
 .../org/apache/kylin/metadata/model/ColumnDesc.java|  8 +---
 2 files changed, 4 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/c804dc8d/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
--
diff --git 
a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java 
b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
index e509d98..a3c89e5 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
@@ -25,22 +25,18 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.measure.MeasureType;
 import org.apache.kylin.measure.basic.BasicMeasureType;
-import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.filter.UDF.MassInTupleFilter;
-import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.IStorageAware;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.ParameterDesc;
-import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.CapabilityResult;
-import org.apache.kylin.metadata.realization.SQLDigest;
 import 
org.apache.kylin.metadata.realization.CapabilityResult.CapabilityInfluence;
+import org.apache.kylin.metadata.realization.SQLDigest;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -86,11 +82,8 @@ public class CubeCapabilityChecker {
 } else {
 //for non query-on-facttable 
 if 
(cube.getSegments().get(0).getSnapshots().containsKey(digest.factTable)) {
-TableDesc tableDesc = 
MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getTableDesc(digest.factTable);
-Set dimCols = Sets.newHashSet();
-for (ColumnDesc columnDesc : tableDesc.getColumns()) {
-dimCols.add(columnDesc.getRef());
-}
+
+Set dimCols = 
Sets.newHashSet(cube.getDataModelDesc().findFirstTable(digest.factTable).getColumns());
 
 //1. all aggregations on lookup table can be done. For 
distinct count, mark them all DimensionAsMeasures
 // so that the measure has a chance to be upgraded to 
DimCountDistinctMeasureType in 
org.apache.kylin.metadata.model.FunctionDesc#reInitMeasureType

http://git-wip-us.apache.org/repos/asf/kylin/blob/c804dc8d/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
--
diff --git 
a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java 
b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
index a13bd37..e0184b4 100644
--- 
a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
+++ 
b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
@@ -26,7 +26,6 @@ import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
 import com.fasterxml.jackson.annotation.JsonInclude;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
 
 /**
  * Column Metadata from Source. All name should be uppercase.
@@ -203,11 +202,6 @@ public class ColumnDesc implements Serializable {
 
 @Override
 public String toString() {
-return "ColumnDesc{" +
-"id='" + id + '\'' +
-", name='" + name + '\'' +
-", datatype='" + datatype + '\'' +
-", comment='" + comment + '\'' +
-'}';
+return "ColumnDesc{" + "id='" + id + '\'' + ", name='" + name + '\'' + 

[6/7] kylin git commit: KYLIN-2030 bug fix

2016-10-26 Thread mahongbin
KYLIN-2030 bug fix


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

Branch: refs/heads/master
Commit: 324e4239c308ba5b14a46eb3187dad3186489f92
Parents: 46284bd
Author: Hongbin Ma 
Authored: Sun Oct 9 19:07:52 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Oct 27 08:30:13 2016 +0800

--
 .../kylin/cube/CubeCapabilityChecker.java   | 23 ++--
 .../kylin/query/relnode/OLAPAggregateRel.java   | 22 +++
 2 files changed, 25 insertions(+), 20 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/324e4239/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
--
diff --git 
a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java 
b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
index ee21b1c..e509d98 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
@@ -81,7 +81,7 @@ public class CubeCapabilityChecker {
 //1. dimension as measure
 
 if (!unmatchedAggregations.isEmpty()) {
-tryDimensionAsMeasures(unmatchedAggregations, digest, cube, 
result, cube.getDescriptor().listDimensionColumnsIncludingDerived());
+tryDimensionAsMeasures(unmatchedAggregations, result, 
cube.getDescriptor().listDimensionColumnsIncludingDerived());
 }
 } else {
 //for non query-on-facttable 
@@ -92,10 +92,18 @@ public class CubeCapabilityChecker {
 dimCols.add(columnDesc.getRef());
 }
 
-//1. dimension as measure, like max(cal_dt) or count( distinct 
col) from lookup
+//1. all aggregations on lookup table can be done. For 
distinct count, mark them all DimensionAsMeasures
+// so that the measure has a chance to be upgraded to 
DimCountDistinctMeasureType in 
org.apache.kylin.metadata.model.FunctionDesc#reInitMeasureType
 if (!unmatchedAggregations.isEmpty()) {
-tryDimensionAsMeasures(unmatchedAggregations, digest, 
cube, result, dimCols);
+Iterator itr = 
unmatchedAggregations.iterator();
+while (itr.hasNext()) {
+FunctionDesc functionDesc = itr.next();
+if 
(dimCols.containsAll(functionDesc.getParameter().getColRefs())) {
+itr.remove();
+}
+}
 }
+tryDimensionAsMeasures(Lists.newArrayList(aggrFunctions), 
result, dimCols);
 
 //2. more "dimensions" contributed by snapshot
 if (!unmatchedDimensions.isEmpty()) {
@@ -159,19 +167,12 @@ public class CubeCapabilityChecker {
 return result;
 }
 
-private static void tryDimensionAsMeasures(Collection 
unmatchedAggregations, SQLDigest digest, CubeInstance cube, CapabilityResult 
result, Set dimCols) {
-CubeDesc cubeDesc = cube.getDescriptor();
-Collection cubeFuncs = cubeDesc.listAllFunctions();
+private static void tryDimensionAsMeasures(Collection 
unmatchedAggregations, CapabilityResult result, Set dimCols) {
 
 Iterator it = unmatchedAggregations.iterator();
 while (it.hasNext()) {
 FunctionDesc functionDesc = it.next();
 
-if (cubeFuncs.contains(functionDesc)) {
-it.remove();
-continue;
-}
-
 // let calcite handle count
 if (functionDesc.isCount()) {
 it.remove();

http://git-wip-us.apache.org/repos/asf/kylin/blob/324e4239/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
--
diff --git 
a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java 
b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
index d73206a..62351d3 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
@@ -296,16 +296,20 @@ public class OLAPAggregateRel extends Aggregate 
implements OLAPRel {
 }
 
 private void translateAggregation() {
-// now the realization is known, replace aggregations with what's 
defined on MeasureDesc
-List measures = this.context.realization.getMeasures();
-List newAggrs = 

[3/7] kylin git commit: port KYLIN-2068 to BeelineHiveClient(KYLIN-2125)

2016-10-26 Thread mahongbin
port KYLIN-2068 to BeelineHiveClient(KYLIN-2125)


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

Branch: refs/heads/master
Commit: ca6837d004c0da31c95027b48db589238179f1ed
Parents: e2a932e
Author: Hongbin Ma 
Authored: Wed Oct 26 13:37:03 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Oct 27 08:30:13 2016 +0800

--
 .../org/apache/kylin/source/hive/BeelineHiveClient.java   | 10 +-
 .../java/org/apache/kylin/source/hive/CLIHiveClient.java  |  4 ++--
 .../apache/kylin/source/hive/HiveSourceTableLoader.java   |  1 +
 .../java/org/apache/kylin/source/hive/HiveTableMeta.java  |  6 --
 4 files changed, 12 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/ca6837d0/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
index 0fbc39b..c8d56a5 100644
--- 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
+++ 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
@@ -115,7 +115,7 @@ public class BeelineHiveClient implements IHiveClient {
 
 List allColumns = 
Lists.newArrayList();
 while (columns.next()) {
-allColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(columns.getString(4), columns.getString(6)));
+allColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(columns.getString(4), columns.getString(6), 
columns.getString(12)));
 }
 builder.setAllColumns(allColumns);
 
@@ -139,7 +139,7 @@ public class BeelineHiveClient implements IHiveClient {
 if ("".equals(resultSet.getString(1).trim())) {
 break;
 }
-partitionColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(resultSet.getString(1).trim(), 
resultSet.getString(2).trim()));
+partitionColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(resultSet.getString(1).trim(), 
resultSet.getString(2).trim(), resultSet.getString(3).trim()));
 }
 builder.setPartitionColumns(partitionColumns);
 }
@@ -205,9 +205,9 @@ public class BeelineHiveClient implements IHiveClient {
 
 public static void main(String[] args) throws SQLException {
 
-//BeelineHiveClient loader = new BeelineHiveClient("-n root --hiveconf 
hive.security.authorization.sqlstd.confwhitelist.append='mapreduce.job.*|dfs.*' 
-u 'jdbc:hive2://sandbox:1'");
-BeelineHiveClient loader = new 
BeelineHiveClient(StringUtils.join(args, " "));
-HiveTableMeta hiveTableMeta = loader.getHiveTableMeta("default", 
"events");
+BeelineHiveClient loader = new BeelineHiveClient("-n root --hiveconf 
hive.security.authorization.sqlstd.confwhitelist.append='mapreduce.job.*|dfs.*' 
-u 'jdbc:hive2://sandbox:1'");
+//BeelineHiveClient loader = new 
BeelineHiveClient(StringUtils.join(args, " "));
+HiveTableMeta hiveTableMeta = loader.getHiveTableMeta("default", 
"test001");
 System.out.println(hiveTableMeta);
 loader.close();
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/ca6837d0/source-hive/src/main/java/org/apache/kylin/source/hive/CLIHiveClient.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/CLIHiveClient.java 
b/source-hive/src/main/java/org/apache/kylin/source/hive/CLIHiveClient.java
index ea74470..60cf47a 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/CLIHiveClient.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/CLIHiveClient.java
@@ -89,11 +89,11 @@ public class CLIHiveClient implements IHiveClient {
 List allColumns = 
Lists.newArrayList();
 List partitionColumns = 
Lists.newArrayList();
 for (FieldSchema fieldSchema : allFields) {
-allColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(fieldSchema.getName(), 
fieldSchema.getType()));
+allColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(fieldSchema.getName(), fieldSchema.getType(), 
fieldSchema.getComment()));
 }
 if (partitionFields != null && partitionFields.size() > 0) {
 for (FieldSchema fieldSchema : partitionFields) {
-partitionColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(fieldSchema.getName(), 

[4/7] kylin git commit: KYLIN-2125 add BeelineHiveClient

2016-10-26 Thread mahongbin
KYLIN-2125 add BeelineHiveClient


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

Branch: refs/heads/master
Commit: e2a932e7439d0034b26b99897d7c293cadb1b8e3
Parents: ea3d02f
Author: Hongbin Ma 
Authored: Thu Oct 13 15:31:11 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Oct 27 08:30:13 2016 +0800

--
 .../kylin/rest/controller/TableController.java  |   9 +-
 source-hive/pom.xml |   5 +
 .../kylin/source/hive/BeelineHiveClient.java| 214 +++
 .../source/hive/BeelineOptionsProcessor.java|  47 
 .../apache/kylin/source/hive/CLIHiveClient.java | 169 +++
 .../apache/kylin/source/hive/HiveClient.java| 170 ---
 .../kylin/source/hive/HiveClientFactory.java|  33 +++
 .../source/hive/HiveSourceTableLoader.java  |  89 
 .../org/apache/kylin/source/hive/HiveTable.java |  16 +-
 .../apache/kylin/source/hive/HiveTableMeta.java |  71 ++
 .../kylin/source/hive/HiveTableMetaBuilder.java | 102 +
 .../apache/kylin/source/hive/HqlExecutable.java | 107 --
 .../apache/kylin/source/hive/IHiveClient.java   |  36 
 .../hive/BeelineOptionsProcessorTest.java   |  38 
 14 files changed, 769 insertions(+), 337 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/e2a932e7/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
--
diff --git 
a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
 
b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
index d9050c1..47ff3fe 100644
--- 
a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
+++ 
b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
@@ -47,7 +47,8 @@ import org.apache.kylin.rest.service.KafkaConfigService;
 import org.apache.kylin.rest.service.ModelService;
 import org.apache.kylin.rest.service.ProjectService;
 import org.apache.kylin.rest.service.StreamingService;
-import org.apache.kylin.source.hive.HiveClient;
+import org.apache.kylin.source.hive.HiveClientFactory;
+import org.apache.kylin.source.hive.IHiveClient;
 import org.apache.kylin.source.kafka.config.KafkaConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -189,7 +190,7 @@ public class TableController extends BasicController {
 String[] dbTableName = HadoopUtil.parseHiveTableName(tableName);
 tableName = dbTableName[0] + "." + dbTableName[1];
 TableDesc desc = 
cubeMgmtService.getMetadataManager().getTableDesc(tableName);
-if(desc == null)
+if (desc == null)
 return false;
 tableType = desc.getSourceType();
 
@@ -315,7 +316,7 @@ public class TableController extends BasicController {
 @RequestMapping(value = "/hive", method = { RequestMethod.GET })
 @ResponseBody
 private static List showHiveDatabases() throws IOException {
-HiveClient hiveClient = new HiveClient();
+IHiveClient hiveClient = HiveClientFactory.getHiveClient();
 List results = null;
 
 try {
@@ -336,7 +337,7 @@ public class TableController extends BasicController {
 @RequestMapping(value = "/hive/{database}", method = { RequestMethod.GET })
 @ResponseBody
 private static List showHiveTables(@PathVariable String database) 
throws IOException {
-HiveClient hiveClient = new HiveClient();
+IHiveClient hiveClient = HiveClientFactory.getHiveClient();
 List results = null;
 
 try {

http://git-wip-us.apache.org/repos/asf/kylin/blob/e2a932e7/source-hive/pom.xml
--
diff --git a/source-hive/pom.xml b/source-hive/pom.xml
index 08019d0..16cb3b4 100644
--- a/source-hive/pom.xml
+++ b/source-hive/pom.xml
@@ -65,6 +65,11 @@
 provided
 
 
+org.apache.hive
+hive-jdbc
+provided
+
+
 org.apache.mrunit
 mrunit
 hadoop2

http://git-wip-us.apache.org/repos/asf/kylin/blob/e2a932e7/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
new file mode 100644
index 000..0fbc39b
--- /dev/null
+++ 

[4/7] kylin git commit: KYLIN-2030 enhancement

2016-10-26 Thread mahongbin
KYLIN-2030 enhancement


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

Branch: refs/heads/orderedbytes4
Commit: 55584c6fef5051b5ef59ab9d3e50a70c94664290
Parents: 2c29fbb
Author: Hongbin Ma 
Authored: Sat Oct 8 16:14:26 2016 +0800
Committer: Hongbin Ma 
Committed: Wed Oct 26 14:42:48 2016 +0800

--
 .../kylin/cube/CubeCapabilityChecker.java   | 43 +++-
 .../org/apache/kylin/cube/model/CubeDesc.java   |  6 +--
 .../kylin/metadata/model/FunctionDesc.java  |  1 +
 .../test/resources/query/sql_lookup/query06.sql |  1 +
 .../test/resources/query/sql_lookup/query07.sql |  1 +
 5 files changed, 39 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/55584c6f/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
--
diff --git 
a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java 
b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
index c8460f0..ee21b1c 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
@@ -25,18 +25,22 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.measure.MeasureType;
 import org.apache.kylin.measure.basic.BasicMeasureType;
+import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.filter.UDF.MassInTupleFilter;
+import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.IStorageAware;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.ParameterDesc;
+import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.CapabilityResult;
-import 
org.apache.kylin.metadata.realization.CapabilityResult.CapabilityInfluence;
 import org.apache.kylin.metadata.realization.SQLDigest;
+import 
org.apache.kylin.metadata.realization.CapabilityResult.CapabilityInfluence;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -66,20 +70,39 @@ public class CubeCapabilityChecker {
 Collection unmatchedAggregations = 
unmatchedAggregations(aggrFunctions, cube);
 
 // try custom measure types
+// in RAW query, unmatchedDimensions and unmatchedAggregations will 
null, so can't chose RAW cube well!
+//if (!unmatchedDimensions.isEmpty() || 
!unmatchedAggregations.isEmpty()) {
 tryCustomMeasureTypes(unmatchedDimensions, unmatchedAggregations, 
digest, cube, result);
+//}
 
-// try dimension-as-measure
-if (!unmatchedAggregations.isEmpty()) {
-if (cube.getDescriptor().getFactTable().equals(digest.factTable)) {
+//more tricks
+if (cube.getDescriptor().getFactTable().equals(digest.factTable)) {
+//for query-on-facttable
+//1. dimension as measure
+
+if (!unmatchedAggregations.isEmpty()) {
 tryDimensionAsMeasures(unmatchedAggregations, digest, cube, 
result, cube.getDescriptor().listDimensionColumnsIncludingDerived());
-} else {
-//deal with query on lookup table, like 
https://issues.apache.org/jira/browse/KYLIN-2030
-if 
(cube.getSegments().get(0).getSnapshots().containsKey(digest.factTable)) {
-Set dimCols = 
Sets.newHashSet(cube.getDataModelDesc().findFirstTable(digest.factTable).getColumns());
+}
+} else {
+//for non query-on-facttable 
+if 
(cube.getSegments().get(0).getSnapshots().containsKey(digest.factTable)) {
+TableDesc tableDesc = 
MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getTableDesc(digest.factTable);
+Set dimCols = Sets.newHashSet();
+for (ColumnDesc columnDesc : tableDesc.getColumns()) {
+dimCols.add(columnDesc.getRef());
+}
+
+//1. dimension as measure, like max(cal_dt) or count( distinct 
col) from lookup
+if (!unmatchedAggregations.isEmpty()) {
 tryDimensionAsMeasures(unmatchedAggregations, digest, 
cube, result, dimCols);
-} else {
-logger.info("Skip 

[5/7] kylin git commit: improve integer type upgrade logic

2016-10-26 Thread mahongbin
improve integer type upgrade logic


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

Branch: refs/heads/orderedbytes4
Commit: 2c29fbb953e1b4f458381f17b6fa368b9493aceb
Parents: 2f7ab44
Author: Hongbin Ma 
Authored: Fri Sep 30 18:31:04 2016 +0800
Committer: Hongbin Ma 
Committed: Wed Oct 26 14:42:48 2016 +0800

--
 .../filter/EvaluatableFunctionTupleFilter.java  | 151 ---
 .../metadata/filter/TupleFilterSerializer.java  |  20 ++-
 .../apache/kylin/metadata/model/ColumnDesc.java |  20 ++-
 .../apache/kylin/metadata/tuple/TupleInfo.java  |   4 +-
 .../apache/kylin/query/schema/OLAPTable.java|  31 +++-
 5 files changed, 63 insertions(+), 163 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/2c29fbb9/core-metadata/src/main/java/org/apache/kylin/metadata/filter/EvaluatableFunctionTupleFilter.java
--
diff --git 
a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/EvaluatableFunctionTupleFilter.java
 
b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/EvaluatableFunctionTupleFilter.java
deleted file mode 100644
index ff24172..000
--- 
a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/EvaluatableFunctionTupleFilter.java
+++ /dev/null
@@ -1,151 +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.
- */
-
-package org.apache.kylin.metadata.filter;
-
-import java.lang.reflect.InvocationTargetException;
-import java.nio.ByteBuffer;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.datatype.DataType;
-import org.apache.kylin.metadata.datatype.StringSerializer;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.metadata.tuple.IEvaluatableTuple;
-
-import com.google.common.collect.Lists;
-
-public class EvaluatableFunctionTupleFilter extends BuiltInFunctionTupleFilter 
{
-
-private boolean constantsInitted = false;
-
-//about non-like
-private List values;
-private Object tupleValue;
-
-public EvaluatableFunctionTupleFilter(String name) {
-super(name, FilterOperatorEnum.EVAL_FUNC);
-values = Lists.newArrayListWithCapacity(1);
-values.add(null);
-}
-
-@Override
-public boolean evaluate(IEvaluatableTuple tuple, IFilterCodeSystem cs) {
-
-// extract tuple value
-Object tupleValue = null;
-for (TupleFilter filter : this.children) {
-if (!isConstant(filter)) {
-filter.evaluate(tuple, cs);
-tupleValue = filter.getValues().iterator().next();
-}
-}
-
-TblColRef tblColRef = this.getColumn();
-DataType strDataType = DataType.getType("string");
-if (tblColRef.getType() != strDataType) {
-throw new IllegalStateException("Only String type is allow in 
BuiltInFunction");
-}
-ByteArray valueByteArray = (ByteArray) tupleValue;
-StringSerializer serializer = new StringSerializer(strDataType);
-String value = 
serializer.deserialize(ByteBuffer.wrap(valueByteArray.array(), 
valueByteArray.offset(), valueByteArray.length()));
-
-try {
-if (isLikeFunction()) {
-return (Boolean) invokeFunction(value);
-} else {
-this.tupleValue = invokeFunction(value);
-//convert back to ByteArray format because the outer 
EvaluatableFunctionTupleFilter assumes input as ByteArray
-ByteBuffer buffer = 
ByteBuffer.allocate(valueByteArray.length() * 2);
-serializer.serialize((String) this.tupleValue, buffer);
-this.tupleValue = new ByteArray(buffer.array(), 0, 
buffer.position());
-
- 

[7/7] kylin git commit: KYLIN-2030 fix with KYLIN-1971

2016-10-26 Thread mahongbin
KYLIN-2030 fix with KYLIN-1971


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

Branch: refs/heads/orderedbytes4
Commit: 601d40a26d20378153f768c803ed33f367e3a93f
Parents: 662bfe9
Author: Hongbin Ma 
Authored: Wed Oct 26 14:41:06 2016 +0800
Committer: Hongbin Ma 
Committed: Wed Oct 26 15:02:49 2016 +0800

--
 .../org/apache/kylin/cube/CubeCapabilityChecker.java   | 13 +++--
 .../org/apache/kylin/metadata/model/ColumnDesc.java|  8 +---
 2 files changed, 4 insertions(+), 17 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/601d40a2/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
--
diff --git 
a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java 
b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
index e509d98..a3c89e5 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
@@ -25,22 +25,18 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.measure.MeasureType;
 import org.apache.kylin.measure.basic.BasicMeasureType;
-import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.filter.UDF.MassInTupleFilter;
-import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.IStorageAware;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.ParameterDesc;
-import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.CapabilityResult;
-import org.apache.kylin.metadata.realization.SQLDigest;
 import 
org.apache.kylin.metadata.realization.CapabilityResult.CapabilityInfluence;
+import org.apache.kylin.metadata.realization.SQLDigest;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -86,11 +82,8 @@ public class CubeCapabilityChecker {
 } else {
 //for non query-on-facttable 
 if 
(cube.getSegments().get(0).getSnapshots().containsKey(digest.factTable)) {
-TableDesc tableDesc = 
MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getTableDesc(digest.factTable);
-Set dimCols = Sets.newHashSet();
-for (ColumnDesc columnDesc : tableDesc.getColumns()) {
-dimCols.add(columnDesc.getRef());
-}
+
+Set dimCols = 
Sets.newHashSet(cube.getDataModelDesc().findFirstTable(digest.factTable).getColumns());
 
 //1. all aggregations on lookup table can be done. For 
distinct count, mark them all DimensionAsMeasures
 // so that the measure has a chance to be upgraded to 
DimCountDistinctMeasureType in 
org.apache.kylin.metadata.model.FunctionDesc#reInitMeasureType

http://git-wip-us.apache.org/repos/asf/kylin/blob/601d40a2/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
--
diff --git 
a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java 
b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
index a13bd37..e0184b4 100644
--- 
a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
+++ 
b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ColumnDesc.java
@@ -26,7 +26,6 @@ import com.fasterxml.jackson.annotation.JsonAutoDetect;
 import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
 import com.fasterxml.jackson.annotation.JsonInclude;
 import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
 
 /**
  * Column Metadata from Source. All name should be uppercase.
@@ -203,11 +202,6 @@ public class ColumnDesc implements Serializable {
 
 @Override
 public String toString() {
-return "ColumnDesc{" +
-"id='" + id + '\'' +
-", name='" + name + '\'' +
-", datatype='" + datatype + '\'' +
-", comment='" + comment + '\'' +
-'}';
+return "ColumnDesc{" + "id='" + id + '\'' + ", name='" + name + '\'' + 
", datatype='" + datatype + '\'' + ", comment='" + comment + '\'' + '}';

[2/7] kylin git commit: port KYLIN-2068 to BeelineHiveClient(KYLIN-2125)

2016-10-26 Thread mahongbin
port KYLIN-2068 to BeelineHiveClient(KYLIN-2125)


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

Branch: refs/heads/orderedbytes4
Commit: 313c47bfba676dad1caca746ba6f34dbb07d8862
Parents: 92a2f1d
Author: Hongbin Ma 
Authored: Wed Oct 26 13:37:03 2016 +0800
Committer: Hongbin Ma 
Committed: Wed Oct 26 14:05:34 2016 +0800

--
 .../org/apache/kylin/source/hive/BeelineHiveClient.java   | 10 +-
 .../java/org/apache/kylin/source/hive/CLIHiveClient.java  |  4 ++--
 .../apache/kylin/source/hive/HiveSourceTableLoader.java   |  1 +
 .../java/org/apache/kylin/source/hive/HiveTableMeta.java  |  6 --
 4 files changed, 12 insertions(+), 9 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/313c47bf/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
index 0fbc39b..c8d56a5 100644
--- 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
+++ 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
@@ -115,7 +115,7 @@ public class BeelineHiveClient implements IHiveClient {
 
 List allColumns = 
Lists.newArrayList();
 while (columns.next()) {
-allColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(columns.getString(4), columns.getString(6)));
+allColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(columns.getString(4), columns.getString(6), 
columns.getString(12)));
 }
 builder.setAllColumns(allColumns);
 
@@ -139,7 +139,7 @@ public class BeelineHiveClient implements IHiveClient {
 if ("".equals(resultSet.getString(1).trim())) {
 break;
 }
-partitionColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(resultSet.getString(1).trim(), 
resultSet.getString(2).trim()));
+partitionColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(resultSet.getString(1).trim(), 
resultSet.getString(2).trim(), resultSet.getString(3).trim()));
 }
 builder.setPartitionColumns(partitionColumns);
 }
@@ -205,9 +205,9 @@ public class BeelineHiveClient implements IHiveClient {
 
 public static void main(String[] args) throws SQLException {
 
-//BeelineHiveClient loader = new BeelineHiveClient("-n root --hiveconf 
hive.security.authorization.sqlstd.confwhitelist.append='mapreduce.job.*|dfs.*' 
-u 'jdbc:hive2://sandbox:1'");
-BeelineHiveClient loader = new 
BeelineHiveClient(StringUtils.join(args, " "));
-HiveTableMeta hiveTableMeta = loader.getHiveTableMeta("default", 
"events");
+BeelineHiveClient loader = new BeelineHiveClient("-n root --hiveconf 
hive.security.authorization.sqlstd.confwhitelist.append='mapreduce.job.*|dfs.*' 
-u 'jdbc:hive2://sandbox:1'");
+//BeelineHiveClient loader = new 
BeelineHiveClient(StringUtils.join(args, " "));
+HiveTableMeta hiveTableMeta = loader.getHiveTableMeta("default", 
"test001");
 System.out.println(hiveTableMeta);
 loader.close();
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/313c47bf/source-hive/src/main/java/org/apache/kylin/source/hive/CLIHiveClient.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/CLIHiveClient.java 
b/source-hive/src/main/java/org/apache/kylin/source/hive/CLIHiveClient.java
index ea74470..60cf47a 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/CLIHiveClient.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/CLIHiveClient.java
@@ -89,11 +89,11 @@ public class CLIHiveClient implements IHiveClient {
 List allColumns = 
Lists.newArrayList();
 List partitionColumns = 
Lists.newArrayList();
 for (FieldSchema fieldSchema : allFields) {
-allColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(fieldSchema.getName(), 
fieldSchema.getType()));
+allColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(fieldSchema.getName(), fieldSchema.getType(), 
fieldSchema.getComment()));
 }
 if (partitionFields != null && partitionFields.size() > 0) {
 for (FieldSchema fieldSchema : partitionFields) {
-partitionColumns.add(new 
HiveTableMeta.HiveTableColumnMeta(fieldSchema.getName(), 

[3/7] kylin git commit: KYLIN-2125 add BeelineHiveClient

2016-10-26 Thread mahongbin
KYLIN-2125 add BeelineHiveClient


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

Branch: refs/heads/orderedbytes4
Commit: 92a2f1dd66a10f7408c1159f256d236c45ba40ca
Parents: 5cdd225
Author: Hongbin Ma 
Authored: Thu Oct 13 15:31:11 2016 +0800
Committer: Hongbin Ma 
Committed: Wed Oct 26 14:05:34 2016 +0800

--
 .../kylin/rest/controller/TableController.java  |   9 +-
 source-hive/pom.xml |   5 +
 .../kylin/source/hive/BeelineHiveClient.java| 214 +++
 .../source/hive/BeelineOptionsProcessor.java|  47 
 .../apache/kylin/source/hive/CLIHiveClient.java | 169 +++
 .../apache/kylin/source/hive/HiveClient.java| 170 ---
 .../kylin/source/hive/HiveClientFactory.java|  33 +++
 .../source/hive/HiveSourceTableLoader.java  |  89 
 .../org/apache/kylin/source/hive/HiveTable.java |  16 +-
 .../apache/kylin/source/hive/HiveTableMeta.java |  71 ++
 .../kylin/source/hive/HiveTableMetaBuilder.java | 102 +
 .../apache/kylin/source/hive/HqlExecutable.java | 107 --
 .../apache/kylin/source/hive/IHiveClient.java   |  36 
 .../hive/BeelineOptionsProcessorTest.java   |  38 
 14 files changed, 769 insertions(+), 337 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/92a2f1dd/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
--
diff --git 
a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
 
b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
index d9050c1..47ff3fe 100644
--- 
a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
+++ 
b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
@@ -47,7 +47,8 @@ import org.apache.kylin.rest.service.KafkaConfigService;
 import org.apache.kylin.rest.service.ModelService;
 import org.apache.kylin.rest.service.ProjectService;
 import org.apache.kylin.rest.service.StreamingService;
-import org.apache.kylin.source.hive.HiveClient;
+import org.apache.kylin.source.hive.HiveClientFactory;
+import org.apache.kylin.source.hive.IHiveClient;
 import org.apache.kylin.source.kafka.config.KafkaConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -189,7 +190,7 @@ public class TableController extends BasicController {
 String[] dbTableName = HadoopUtil.parseHiveTableName(tableName);
 tableName = dbTableName[0] + "." + dbTableName[1];
 TableDesc desc = 
cubeMgmtService.getMetadataManager().getTableDesc(tableName);
-if(desc == null)
+if (desc == null)
 return false;
 tableType = desc.getSourceType();
 
@@ -315,7 +316,7 @@ public class TableController extends BasicController {
 @RequestMapping(value = "/hive", method = { RequestMethod.GET })
 @ResponseBody
 private static List showHiveDatabases() throws IOException {
-HiveClient hiveClient = new HiveClient();
+IHiveClient hiveClient = HiveClientFactory.getHiveClient();
 List results = null;
 
 try {
@@ -336,7 +337,7 @@ public class TableController extends BasicController {
 @RequestMapping(value = "/hive/{database}", method = { RequestMethod.GET })
 @ResponseBody
 private static List showHiveTables(@PathVariable String database) 
throws IOException {
-HiveClient hiveClient = new HiveClient();
+IHiveClient hiveClient = HiveClientFactory.getHiveClient();
 List results = null;
 
 try {

http://git-wip-us.apache.org/repos/asf/kylin/blob/92a2f1dd/source-hive/pom.xml
--
diff --git a/source-hive/pom.xml b/source-hive/pom.xml
index 08019d0..16cb3b4 100644
--- a/source-hive/pom.xml
+++ b/source-hive/pom.xml
@@ -65,6 +65,11 @@
 provided
 
 
+org.apache.hive
+hive-jdbc
+provided
+
+
 org.apache.mrunit
 mrunit
 hadoop2

http://git-wip-us.apache.org/repos/asf/kylin/blob/92a2f1dd/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
new file mode 100644
index 000..0fbc39b
--- /dev/null
+++ 

[6/7] kylin git commit: KYLIN-2030 bug fix

2016-10-26 Thread mahongbin
KYLIN-2030 bug fix


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

Branch: refs/heads/orderedbytes4
Commit: 662bfe918e0fa2f3f195acfda815330dbcb32f21
Parents: 55584c6
Author: Hongbin Ma 
Authored: Sun Oct 9 19:07:52 2016 +0800
Committer: Hongbin Ma 
Committed: Wed Oct 26 14:42:48 2016 +0800

--
 .../kylin/cube/CubeCapabilityChecker.java   | 23 ++--
 .../kylin/query/relnode/OLAPAggregateRel.java   | 22 +++
 2 files changed, 25 insertions(+), 20 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/662bfe91/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
--
diff --git 
a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java 
b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
index ee21b1c..e509d98 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
@@ -81,7 +81,7 @@ public class CubeCapabilityChecker {
 //1. dimension as measure
 
 if (!unmatchedAggregations.isEmpty()) {
-tryDimensionAsMeasures(unmatchedAggregations, digest, cube, 
result, cube.getDescriptor().listDimensionColumnsIncludingDerived());
+tryDimensionAsMeasures(unmatchedAggregations, result, 
cube.getDescriptor().listDimensionColumnsIncludingDerived());
 }
 } else {
 //for non query-on-facttable 
@@ -92,10 +92,18 @@ public class CubeCapabilityChecker {
 dimCols.add(columnDesc.getRef());
 }
 
-//1. dimension as measure, like max(cal_dt) or count( distinct 
col) from lookup
+//1. all aggregations on lookup table can be done. For 
distinct count, mark them all DimensionAsMeasures
+// so that the measure has a chance to be upgraded to 
DimCountDistinctMeasureType in 
org.apache.kylin.metadata.model.FunctionDesc#reInitMeasureType
 if (!unmatchedAggregations.isEmpty()) {
-tryDimensionAsMeasures(unmatchedAggregations, digest, 
cube, result, dimCols);
+Iterator itr = 
unmatchedAggregations.iterator();
+while (itr.hasNext()) {
+FunctionDesc functionDesc = itr.next();
+if 
(dimCols.containsAll(functionDesc.getParameter().getColRefs())) {
+itr.remove();
+}
+}
 }
+tryDimensionAsMeasures(Lists.newArrayList(aggrFunctions), 
result, dimCols);
 
 //2. more "dimensions" contributed by snapshot
 if (!unmatchedDimensions.isEmpty()) {
@@ -159,19 +167,12 @@ public class CubeCapabilityChecker {
 return result;
 }
 
-private static void tryDimensionAsMeasures(Collection 
unmatchedAggregations, SQLDigest digest, CubeInstance cube, CapabilityResult 
result, Set dimCols) {
-CubeDesc cubeDesc = cube.getDescriptor();
-Collection cubeFuncs = cubeDesc.listAllFunctions();
+private static void tryDimensionAsMeasures(Collection 
unmatchedAggregations, CapabilityResult result, Set dimCols) {
 
 Iterator it = unmatchedAggregations.iterator();
 while (it.hasNext()) {
 FunctionDesc functionDesc = it.next();
 
-if (cubeFuncs.contains(functionDesc)) {
-it.remove();
-continue;
-}
-
 // let calcite handle count
 if (functionDesc.isCount()) {
 it.remove();

http://git-wip-us.apache.org/repos/asf/kylin/blob/662bfe91/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
--
diff --git 
a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java 
b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
index d73206a..62351d3 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
@@ -296,16 +296,20 @@ public class OLAPAggregateRel extends Aggregate 
implements OLAPRel {
 }
 
 private void translateAggregation() {
-// now the realization is known, replace aggregations with what's 
defined on MeasureDesc
-List measures = this.context.realization.getMeasures();
-List newAggrs = 

[1/7] kylin git commit: port KYLIN-2012 to new interface introduced in KYLIN-2125

2016-10-26 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/orderedbytes4 [created] 601d40a26


port KYLIN-2012 to new interface introduced in KYLIN-2125


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

Branch: refs/heads/orderedbytes4
Commit: 2f7ab44d641d73b333c7e05bc27e1baa88b911c4
Parents: 313c47b
Author: Hongbin Ma 
Authored: Wed Oct 26 14:04:56 2016 +0800
Committer: Hongbin Ma 
Committed: Wed Oct 26 14:05:34 2016 +0800

--
 .../java/org/apache/kylin/job/DeployUtil.java   |  5 ++-
 .../kylin/source/hive/BeelineHiveClient.java|  2 +-
 .../source/hive/HiveSourceTableLoader.java  | 32 +++
 .../apache/kylin/source/hive/SchemaChecker.java | 41 
 4 files changed, 36 insertions(+), 44 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/2f7ab44d/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
--
diff --git a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java 
b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
index 4a24ad2..54feb24 100644
--- a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -45,8 +45,9 @@ import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TableRef;
 import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.source.hive.HiveClient;
+import org.apache.kylin.source.hive.HiveClientFactory;
 import org.apache.kylin.source.hive.HiveCmdBuilder;
+import org.apache.kylin.source.hive.IHiveClient;
 import org.apache.kylin.source.kafka.TimedJsonStreamParser;
 import org.apache.maven.model.Model;
 import org.apache.maven.model.io.xpp3.MavenXpp3Reader;
@@ -234,7 +235,7 @@ public class DeployUtil {
 String tableFileDir = temp.getParent();
 temp.delete();
 
-HiveClient hiveClient = new HiveClient();
+IHiveClient hiveClient = HiveClientFactory.getHiveClient();
 // create hive tables
 hiveClient.executeHQL("CREATE DATABASE IF NOT EXISTS EDW");
 
hiveClient.executeHQL(generateCreateTableHql(metaMgr.getTableDesc(TABLE_CAL_DT.toUpperCase(;

http://git-wip-us.apache.org/repos/asf/kylin/blob/2f7ab44d/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
index c8d56a5..a84aeb1 100644
--- 
a/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
+++ 
b/source-hive/src/main/java/org/apache/kylin/source/hive/BeelineHiveClient.java
@@ -207,7 +207,7 @@ public class BeelineHiveClient implements IHiveClient {
 
 BeelineHiveClient loader = new BeelineHiveClient("-n root --hiveconf 
hive.security.authorization.sqlstd.confwhitelist.append='mapreduce.job.*|dfs.*' 
-u 'jdbc:hive2://sandbox:1'");
 //BeelineHiveClient loader = new 
BeelineHiveClient(StringUtils.join(args, " "));
-HiveTableMeta hiveTableMeta = loader.getHiveTableMeta("default", 
"test001");
+HiveTableMeta hiveTableMeta = loader.getHiveTableMeta("default", 
"test_kylin_fact_part");
 System.out.println(hiveTableMeta);
 loader.close();
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/2f7ab44d/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
--
diff --git 
a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
 
b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
index 388e72b..401e720 100644
--- 
a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
+++ 
b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveSourceTableLoader.java
@@ -26,6 +26,7 @@ import java.util.Set;
 import java.util.UUID;
 
 import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.cube.CubeManager;
 import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.metadata.MetadataConstants;
 import org.apache.kylin.metadata.MetadataManager;
@@ -34,8 +35,10 @@ import org.apache.kylin.metadata.model.TableDesc;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.LinkedHashMultimap;
 import com.google.common.collect.Lists;
 

[5/5] kylin git commit: improve integer type upgrade logic

2016-10-15 Thread mahongbin
improve integer type upgrade logic


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

Branch: refs/heads/orderedbytes3
Commit: 3d0954d1e902755ce6f33719c4e74a368ee537d4
Parents: c60c2d6
Author: Hongbin Ma 
Authored: Fri Sep 30 18:31:04 2016 +0800
Committer: Hongbin Ma 
Committed: Sun Oct 16 13:27:26 2016 +0800

--
 .../filter/EvaluatableFunctionTupleFilter.java  | 151 ---
 .../metadata/filter/TupleFilterSerializer.java  |  20 ++-
 .../apache/kylin/metadata/model/ColumnDesc.java |  21 ++-
 .../apache/kylin/metadata/tuple/TupleInfo.java  |   4 +-
 .../apache/kylin/query/schema/OLAPTable.java|  31 +++-
 5 files changed, 63 insertions(+), 164 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/3d0954d1/core-metadata/src/main/java/org/apache/kylin/metadata/filter/EvaluatableFunctionTupleFilter.java
--
diff --git 
a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/EvaluatableFunctionTupleFilter.java
 
b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/EvaluatableFunctionTupleFilter.java
deleted file mode 100644
index ff24172..000
--- 
a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/EvaluatableFunctionTupleFilter.java
+++ /dev/null
@@ -1,151 +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.
- */
-
-package org.apache.kylin.metadata.filter;
-
-import java.lang.reflect.InvocationTargetException;
-import java.nio.ByteBuffer;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.datatype.DataType;
-import org.apache.kylin.metadata.datatype.StringSerializer;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.metadata.tuple.IEvaluatableTuple;
-
-import com.google.common.collect.Lists;
-
-public class EvaluatableFunctionTupleFilter extends BuiltInFunctionTupleFilter 
{
-
-private boolean constantsInitted = false;
-
-//about non-like
-private List values;
-private Object tupleValue;
-
-public EvaluatableFunctionTupleFilter(String name) {
-super(name, FilterOperatorEnum.EVAL_FUNC);
-values = Lists.newArrayListWithCapacity(1);
-values.add(null);
-}
-
-@Override
-public boolean evaluate(IEvaluatableTuple tuple, IFilterCodeSystem cs) {
-
-// extract tuple value
-Object tupleValue = null;
-for (TupleFilter filter : this.children) {
-if (!isConstant(filter)) {
-filter.evaluate(tuple, cs);
-tupleValue = filter.getValues().iterator().next();
-}
-}
-
-TblColRef tblColRef = this.getColumn();
-DataType strDataType = DataType.getType("string");
-if (tblColRef.getType() != strDataType) {
-throw new IllegalStateException("Only String type is allow in 
BuiltInFunction");
-}
-ByteArray valueByteArray = (ByteArray) tupleValue;
-StringSerializer serializer = new StringSerializer(strDataType);
-String value = 
serializer.deserialize(ByteBuffer.wrap(valueByteArray.array(), 
valueByteArray.offset(), valueByteArray.length()));
-
-try {
-if (isLikeFunction()) {
-return (Boolean) invokeFunction(value);
-} else {
-this.tupleValue = invokeFunction(value);
-//convert back to ByteArray format because the outer 
EvaluatableFunctionTupleFilter assumes input as ByteArray
-ByteBuffer buffer = 
ByteBuffer.allocate(valueByteArray.length() * 2);
-serializer.serialize((String) this.tupleValue, buffer);
-this.tupleValue = new ByteArray(buffer.array(), 0, 
buffer.position());
-
- 

[4/5] kylin git commit: KYLIN-2030 enhancement

2016-10-15 Thread mahongbin
KYLIN-2030 enhancement


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

Branch: refs/heads/orderedbytes3
Commit: ae73be5204452face5f28a6b5799de2e22b0a198
Parents: 8f1fb6a
Author: Hongbin Ma 
Authored: Sat Oct 8 16:14:26 2016 +0800
Committer: Hongbin Ma 
Committed: Sun Oct 16 13:27:26 2016 +0800

--
 .../kylin/cube/CubeCapabilityChecker.java   | 38 +---
 .../org/apache/kylin/cube/model/CubeDesc.java   | 10 +++---
 .../kylin/metadata/model/FunctionDesc.java  |  1 +
 .../test/resources/query/sql_lookup/query06.sql |  1 +
 .../test/resources/query/sql_lookup/query07.sql |  1 +
 5 files changed, 32 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/ae73be52/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
--
diff --git 
a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java 
b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
index 1eada16..ee21b1c 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
@@ -75,22 +75,34 @@ public class CubeCapabilityChecker {
 tryCustomMeasureTypes(unmatchedDimensions, unmatchedAggregations, 
digest, cube, result);
 //}
 
-// try dimension-as-measure
-if (!unmatchedAggregations.isEmpty()) {
-if (cube.getDescriptor().getFactTable().equals(digest.factTable)) {
+//more tricks
+if (cube.getDescriptor().getFactTable().equals(digest.factTable)) {
+//for query-on-facttable
+//1. dimension as measure
+
+if (!unmatchedAggregations.isEmpty()) {
 tryDimensionAsMeasures(unmatchedAggregations, digest, cube, 
result, cube.getDescriptor().listDimensionColumnsIncludingDerived());
-} else {
-//deal with query on lookup table, like 
https://issues.apache.org/jira/browse/KYLIN-2030
-if 
(cube.getSegments().get(0).getSnapshots().containsKey(digest.factTable)) {
-TableDesc tableDesc = 
MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getTableDesc(digest.factTable);
-Set dimCols = Sets.newHashSet();
-for (ColumnDesc columnDesc : tableDesc.getColumns()) {
-dimCols.add(columnDesc.getRef());
-}
+}
+} else {
+//for non query-on-facttable 
+if 
(cube.getSegments().get(0).getSnapshots().containsKey(digest.factTable)) {
+TableDesc tableDesc = 
MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getTableDesc(digest.factTable);
+Set dimCols = Sets.newHashSet();
+for (ColumnDesc columnDesc : tableDesc.getColumns()) {
+dimCols.add(columnDesc.getRef());
+}
+
+//1. dimension as measure, like max(cal_dt) or count( distinct 
col) from lookup
+if (!unmatchedAggregations.isEmpty()) {
 tryDimensionAsMeasures(unmatchedAggregations, digest, 
cube, result, dimCols);
-} else {
-logger.info("Skip tryDimensionAsMeasures because current 
cube {} does not touch lookup table {} at all", cube.getName(), 
digest.factTable);
 }
+
+//2. more "dimensions" contributed by snapshot
+if (!unmatchedDimensions.isEmpty()) {
+unmatchedDimensions.removeAll(dimCols);
+}
+} else {
+logger.info("cube {} does not touch lookup table {} at all", 
cube.getName(), digest.factTable);
 }
 }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/ae73be52/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 4195451..7e055be 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
@@ -33,9 +33,9 @@ import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.Map.Entry;
 
 

[1/5] kylin git commit: KYLIN-2030 bug fix

2016-10-15 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/orderedbytes3 [created] 7bce09614


KYLIN-2030 bug fix


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

Branch: refs/heads/orderedbytes3
Commit: 7bce09614abfeea7c567c047929a9c985f5094d0
Parents: ae73be5
Author: Hongbin Ma 
Authored: Sun Oct 9 19:07:52 2016 +0800
Committer: Hongbin Ma 
Committed: Sun Oct 16 13:27:26 2016 +0800

--
 .../kylin/cube/CubeCapabilityChecker.java   | 23 ++--
 .../kylin/query/relnode/OLAPAggregateRel.java   | 22 +++
 2 files changed, 25 insertions(+), 20 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/7bce0961/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
--
diff --git 
a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java 
b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
index ee21b1c..e509d98 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
@@ -81,7 +81,7 @@ public class CubeCapabilityChecker {
 //1. dimension as measure
 
 if (!unmatchedAggregations.isEmpty()) {
-tryDimensionAsMeasures(unmatchedAggregations, digest, cube, 
result, cube.getDescriptor().listDimensionColumnsIncludingDerived());
+tryDimensionAsMeasures(unmatchedAggregations, result, 
cube.getDescriptor().listDimensionColumnsIncludingDerived());
 }
 } else {
 //for non query-on-facttable 
@@ -92,10 +92,18 @@ public class CubeCapabilityChecker {
 dimCols.add(columnDesc.getRef());
 }
 
-//1. dimension as measure, like max(cal_dt) or count( distinct 
col) from lookup
+//1. all aggregations on lookup table can be done. For 
distinct count, mark them all DimensionAsMeasures
+// so that the measure has a chance to be upgraded to 
DimCountDistinctMeasureType in 
org.apache.kylin.metadata.model.FunctionDesc#reInitMeasureType
 if (!unmatchedAggregations.isEmpty()) {
-tryDimensionAsMeasures(unmatchedAggregations, digest, 
cube, result, dimCols);
+Iterator itr = 
unmatchedAggregations.iterator();
+while (itr.hasNext()) {
+FunctionDesc functionDesc = itr.next();
+if 
(dimCols.containsAll(functionDesc.getParameter().getColRefs())) {
+itr.remove();
+}
+}
 }
+tryDimensionAsMeasures(Lists.newArrayList(aggrFunctions), 
result, dimCols);
 
 //2. more "dimensions" contributed by snapshot
 if (!unmatchedDimensions.isEmpty()) {
@@ -159,19 +167,12 @@ public class CubeCapabilityChecker {
 return result;
 }
 
-private static void tryDimensionAsMeasures(Collection 
unmatchedAggregations, SQLDigest digest, CubeInstance cube, CapabilityResult 
result, Set dimCols) {
-CubeDesc cubeDesc = cube.getDescriptor();
-Collection cubeFuncs = cubeDesc.listAllFunctions();
+private static void tryDimensionAsMeasures(Collection 
unmatchedAggregations, CapabilityResult result, Set dimCols) {
 
 Iterator it = unmatchedAggregations.iterator();
 while (it.hasNext()) {
 FunctionDesc functionDesc = it.next();
 
-if (cubeFuncs.contains(functionDesc)) {
-it.remove();
-continue;
-}
-
 // let calcite handle count
 if (functionDesc.isCount()) {
 it.remove();

http://git-wip-us.apache.org/repos/asf/kylin/blob/7bce0961/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
--
diff --git 
a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java 
b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
index 97efb27..c7a1eff 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
@@ -285,16 +285,20 @@ public class OLAPAggregateRel extends Aggregate 
implements OLAPRel {
 }
 
 private void translateAggregation() {
-// now the realization is known, replace aggregations with what's 
defined on MeasureDesc
- 

[2/5] kylin git commit: KYLIN-1726 fix BuildCubeWithStream

2016-10-15 Thread mahongbin
KYLIN-1726 fix BuildCubeWithStream


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

Branch: refs/heads/orderedbytes3
Commit: 8f1fb6ac47b9fcf22090080b86d2ea98cc9d6132
Parents: 3d0954d
Author: Hongbin Ma 
Authored: Sat Oct 8 14:08:41 2016 +0800
Committer: Hongbin Ma 
Committed: Sun Oct 16 13:27:26 2016 +0800

--
 .../java/org/apache/kylin/provision/BuildCubeWithStream.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/8f1fb6ac/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
--
diff --git 
a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java 
b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
index fed14d8..c2fb180 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
@@ -18,6 +18,8 @@
 
 package org.apache.kylin.provision;
 
+import static java.lang.Thread.sleep;
+
 import java.io.File;
 import java.io.IOException;
 import java.text.ParseException;
@@ -32,7 +34,6 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.FutureTask;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.collect.Lists;
 import org.I0Itec.zkclient.ZkConnection;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.kafka.common.requests.MetadataResponse;
@@ -64,7 +65,7 @@ import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static java.lang.Thread.sleep;
+import com.google.common.collect.Lists;
 
 /**
  *  for streaming cubing case "test_streaming_table"
@@ -323,6 +324,5 @@ public class BuildCubeWithStream {
 logger.error("error", e);
 System.exit(1);
 }
-
 }
 }



[3/5] kylin git commit: invoke after() in buildcubewithstream

2016-10-15 Thread mahongbin
invoke after() in buildcubewithstream


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

Branch: refs/heads/orderedbytes3
Commit: c60c2d628ed0a3a4b852129727aa8f42d77c
Parents: 61a08d4
Author: Hongbin Ma 
Authored: Sun Oct 16 13:25:33 2016 +0800
Committer: Hongbin Ma 
Committed: Sun Oct 16 13:27:26 2016 +0800

--
 .../test/java/org/apache/kylin/provision/BuildCubeWithStream.java   | 1 +
 1 file changed, 1 insertion(+)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/c60c2d62/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
--
diff --git 
a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java 
b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
index 2faa8d0..fed14d8 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
@@ -316,6 +316,7 @@ public class BuildCubeWithStream {
 buildCubeWithStream.build();
 logger.info("Build is done");
 buildCubeWithStream.cleanup();
+buildCubeWithStream.after();
 logger.info("Going to exit");
 System.exit(0);
 } catch (Throwable e) {



[06/10] kylin git commit: KYLIN-1528 Create a branch for v1.5 with HBase 1.x API

2016-10-14 Thread mahongbin
http://git-wip-us.apache.org/repos/asf/kylin/blob/c4e31c1b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
--
diff --git 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
index 9b487a7..4a4f2a3 100644
--- 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
+++ 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/CubeHTableUtil.java
@@ -25,7 +25,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.regionserver.BloomType;
@@ -78,7 +79,8 @@ public class CubeHTableUtil {
 tableDesc.setValue(IRealizationConstants.HTableSegmentTag, 
cubeSegment.toString());
 
 Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-HBaseAdmin admin = new HBaseAdmin(conf);
+Connection conn = HBaseConnection.get(kylinConfig.getStorageUrl());
+Admin admin = conn.getAdmin();
 
 try {
 if (User.isHBaseSecurityEnabled(conf)) {
@@ -91,7 +93,7 @@ public class CubeHTableUtil {
 tableDesc.addFamily(cf);
 }
 
-if (admin.tableExists(tableName)) {
+if (admin.tableExists(TableName.valueOf(tableName))) {
 // admin.disableTable(tableName);
 // admin.deleteTable(tableName);
 throw new RuntimeException("HBase table " + tableName + " 
exists!");
@@ -100,7 +102,7 @@ public class CubeHTableUtil {
 DeployCoprocessorCLI.deployCoprocessor(tableDesc);
 
 admin.createTable(tableDesc, splitKeys);
-Preconditions.checkArgument(admin.isTableAvailable(tableName), 
"table " + tableName + " created, but is not available due to some reasons");
+
Preconditions.checkArgument(admin.isTableAvailable(TableName.valueOf(tableName)),
 "table " + tableName + " created, but is not available due to some reasons");
 logger.info("create hbase table " + tableName + " done.");
 } finally {
 admin.close();
@@ -109,8 +111,7 @@ public class CubeHTableUtil {
 }
 
 public static void deleteHTable(TableName tableName) throws IOException {
-Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-HBaseAdmin admin = new HBaseAdmin(conf);
+Admin admin = 
HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl()).getAdmin();
 try {
 if (admin.tableExists(tableName)) {
 logger.info("disabling hbase table " + tableName);
@@ -125,8 +126,7 @@ public class CubeHTableUtil {
 
 /** create a HTable that has the same performance settings as normal cube 
table, for benchmark purpose */
 public static void createBenchmarkHTable(TableName tableName, String 
cfName) throws IOException {
-Configuration conf = HBaseConnection.getCurrentHBaseConfiguration();
-HBaseAdmin admin = new HBaseAdmin(conf);
+Admin admin = 
HBaseConnection.get(KylinConfig.getInstanceFromEnv().getStorageUrl()).getAdmin();
 try {
 if (admin.tableExists(tableName)) {
 logger.info("disabling hbase table " + tableName);

http://git-wip-us.apache.org/repos/asf/kylin/blob/c4e31c1b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
--
diff --git 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
index 7aecd7e..9dc9715 100644
--- 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
+++ 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/DeprecatedGCStep.java
@@ -28,9 +28,10 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
 import org.apache.kylin.common.KylinConfig;
-import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.engine.mr.HadoopUtil;
 import org.apache.kylin.job.exception.ExecuteException;
 import 

[05/10] kylin git commit: KYLIN-1528 Create a branch for v1.5 with HBase 1.x API

2016-10-14 Thread mahongbin
http://git-wip-us.apache.org/repos/asf/kylin/blob/c4e31c1b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
--
diff --git 
a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
 
b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
index 1d85922..04e2e8b 100644
--- 
a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
+++ 
b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/cube/v1/filter/TestFuzzyRowFilterV2EndToEnd.java
@@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FilterList.Operator;
 import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
 import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -136,7 +137,7 @@ public class TestFuzzyRowFilterV2EndToEnd {
 
 Put p = new Put(rk);
 p.setDurability(Durability.SKIP_WAL);
-p.add(cf.getBytes(), cq, Bytes.toBytes(c));
+p.addColumn(cf.getBytes(), cq, Bytes.toBytes(c));
 ht.put(p);
 }
 }
@@ -224,7 +225,7 @@ public class TestFuzzyRowFilterV2EndToEnd {
 scan.addFamily(cf.getBytes());
 scan.setFilter(filter);
 List regions = 
TEST_UTIL.getHBaseCluster().getRegions(table.getBytes());
-HRegion first = regions.get(0);
+Region first = regions.get(0);
 first.getScanner(scan);
 RegionScanner scanner = first.getScanner(scan);
 List results = new ArrayList();



[09/10] kylin git commit: BeelineHiveClient.java

2016-10-14 Thread mahongbin
BeelineHiveClient.java


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

Branch: refs/heads/1.5.4.1-beeline
Commit: 13b7b542d8fdf45de6aa7bcbeaf9a089bbc9
Parents: 3bd0c10
Author: Hongbin Ma 
Authored: Thu Oct 13 15:31:11 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Oct 13 15:31:11 2016 +0800

--
 .../java/org/apache/kylin/job/DeployUtil.java   |  16 +-
 .../kylin/rest/controller/TableController.java  |   9 +-
 source-hive/pom.xml |   5 +
 .../kylin/source/hive/BeelineHiveClient.java| 214 +++
 .../source/hive/BeelineOptionsProcessor.java|  47 
 .../apache/kylin/source/hive/CLIHiveClient.java | 169 +++
 .../apache/kylin/source/hive/HiveClient.java| 170 ---
 .../kylin/source/hive/HiveClientFactory.java|  33 +++
 .../source/hive/HiveSourceTableLoader.java  |  57 ++---
 .../org/apache/kylin/source/hive/HiveTable.java |  16 +-
 .../apache/kylin/source/hive/HiveTableMeta.java |  71 ++
 .../kylin/source/hive/HiveTableMetaBuilder.java | 102 +
 .../apache/kylin/source/hive/HqlExecutable.java | 107 --
 .../apache/kylin/source/hive/IHiveClient.java   |  36 
 .../hive/BeelineOptionsProcessorTest.java   |  38 
 15 files changed, 762 insertions(+), 328 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/13b7b542/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
--
diff --git a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java 
b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
index 8c64f91..7a8cbe4 100644
--- a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -18,7 +18,12 @@
 
 package org.apache.kylin.job;
 
-import java.io.*;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.io.InputStream;
 import java.nio.ByteBuffer;
 import java.util.List;
 
@@ -38,8 +43,9 @@ import org.apache.kylin.metadata.MetadataManager;
 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.hive.HiveClient;
+import org.apache.kylin.source.hive.HiveClientFactory;
 import org.apache.kylin.source.hive.HiveCmdBuilder;
+import org.apache.kylin.source.hive.IHiveClient;
 import org.apache.kylin.source.kafka.TimedJsonStreamParser;
 import org.apache.maven.model.Model;
 import org.apache.maven.model.io.xpp3.MavenXpp3Reader;
@@ -48,9 +54,6 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Lists;
 
-import kafka.message.Message;
-import kafka.message.MessageAndOffset;
-
 public class DeployUtil {
 private static final Logger logger = 
LoggerFactory.getLogger(DeployUtil.class);
 
@@ -179,7 +182,6 @@ public class DeployUtil {
 in.close();
 }
 
-
 private static void deployHiveTables() throws Exception {
 
 MetadataManager metaMgr = MetadataManager.getInstance(config());
@@ -205,7 +207,7 @@ public class DeployUtil {
 String tableFileDir = temp.getParent();
 temp.delete();
 
-HiveClient hiveClient = new HiveClient();
+IHiveClient hiveClient = HiveClientFactory.getHiveClient();
 // create hive tables
 hiveClient.executeHQL("CREATE DATABASE IF NOT EXISTS EDW");
 
hiveClient.executeHQL(generateCreateTableHql(metaMgr.getTableDesc(TABLE_CAL_DT.toUpperCase(;

http://git-wip-us.apache.org/repos/asf/kylin/blob/13b7b542/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
--
diff --git 
a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
 
b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
index eefeba8..b7dd728 100644
--- 
a/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
+++ 
b/server-base/src/main/java/org/apache/kylin/rest/controller/TableController.java
@@ -47,7 +47,8 @@ import org.apache.kylin.rest.service.KafkaConfigService;
 import org.apache.kylin.rest.service.ModelService;
 import org.apache.kylin.rest.service.ProjectService;
 import org.apache.kylin.rest.service.StreamingService;
-import org.apache.kylin.source.hive.HiveClient;
+import org.apache.kylin.source.hive.HiveClientFactory;
+import 

[02/10] kylin git commit: Revert "KYLIN-2012 more robust approach to hive schema changes"

2016-10-14 Thread mahongbin
Revert "KYLIN-2012 more robust approach to hive schema changes"

This reverts commit 17569f6c32a373f599ef7689f9506b5af5ed68bd.


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

Branch: refs/heads/1.5.4.1-beeline
Commit: d33ba6aaaddf526b78225eab2a9280a39dd2058d
Parents: 18a1454
Author: shaofengshi 
Authored: Fri Sep 23 18:11:29 2016 +0800
Committer: shaofengshi 
Committed: Fri Sep 23 18:11:29 2016 +0800

--
 .../org/apache/kylin/cube/CubeDescManager.java  |  62 +++---
 .../org/apache/kylin/cube/CubeInstance.java |  11 +-
 .../java/org/apache/kylin/cube/CubeManager.java |  47 ++--
 .../org/apache/kylin/cube/model/CubeDesc.java   |  47 ++--
 .../model/validation/CubeMetadataValidator.java |  32 ++-
 .../realization/RealizationStatusEnum.java  |   2 +-
 .../kylin/rest/controller/CubeController.java   |  44 ++--
 .../apache/kylin/rest/service/CacheService.java |  11 +-
 .../apache/kylin/rest/service/CubeService.java  |  15 ++
 .../apache/kylin/rest/service/JobService.java   |   6 -
 .../kylin/rest/service/CubeServiceTest.java |   1 +
 .../source/hive/HiveSourceTableLoader.java  |  33 +--
 .../apache/kylin/source/hive/SchemaChecker.java | 216 ---
 webapp/app/css/AdminLTE.css |   4 +-
 webapp/app/partials/cubes/cubes.html|  22 +-
 15 files changed, 200 insertions(+), 353 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/d33ba6aa/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 1b1cf70..33a6830 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
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.persistence.JsonSerializer;
 import org.apache.kylin.common.persistence.ResourceStore;
@@ -35,7 +36,6 @@ 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.realization.RealizationStatusEnum;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -110,34 +110,30 @@ public class CubeDescManager {
  * @throws IOException
  */
 public CubeDesc reloadCubeDescLocal(String name) throws IOException {
-// Broken CubeDesc is not allowed to be saved and broadcast.
-CubeDesc ndesc = loadCubeDesc(CubeDesc.concatResourcePath(name), 
false);
 
-cubeDescMap.putLocal(ndesc.getName(), ndesc);
-Cuboid.reloadCache(name);
+// Save Source
+String path = CubeDesc.concatResourcePath(name);
 
-// if related cube is in DESCBROKEN state before, change it back to 
DISABLED
-CubeManager cubeManager = CubeManager.getInstance(config);
-for (CubeInstance cube : cubeManager.getCubesByDesc(name)) {
-if (cube.getStatus() == RealizationStatusEnum.DESCBROKEN) {
-cubeManager.reloadCubeLocal(cube.getName());
-}
-}
+// Reload the CubeDesc
+CubeDesc ndesc = loadCubeDesc(path);
 
+// Here replace the old one
+cubeDescMap.putLocal(ndesc.getName(), ndesc);
+Cuboid.reloadCache(name);
 return ndesc;
 }
 
-private CubeDesc loadCubeDesc(String path, boolean allowBroken) throws 
IOException {
+private CubeDesc loadCubeDesc(String path) throws IOException {
 ResourceStore store = getStore();
 CubeDesc ndesc = store.getResource(path, CubeDesc.class, 
CUBE_DESC_SERIALIZER);
 
-try {
-ndesc.init(config, getMetadataManager().getAllTablesMap());
-} catch (Exception e) {
-ndesc.addError(e.getMessage());
+if (StringUtils.isBlank(ndesc.getName())) {
+throw new IllegalStateException("CubeDesc name must not be blank");
 }
 
-if (!allowBroken && !ndesc.getError().isEmpty()) {
+ndesc.init(config, getMetadataManager().getAllTablesMap());
+
+if (ndesc.getError().isEmpty() == false) {
 throw new IllegalStateException("Cube desc at " + path + " has 
issues: " 

[07/10] kylin git commit: KYLIN-1528 Create a branch for v1.5 with HBase 1.x API

2016-10-14 Thread mahongbin
KYLIN-1528 Create a branch for v1.5 with HBase 1.x API


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

Branch: refs/heads/1.5.4.1-beeline
Commit: c4e31c1b3a664f598352061ae8703812e9d9cef7
Parents: b08871e
Author: shaofengshi 
Authored: Wed Mar 23 17:07:05 2016 +0800
Committer: shaofengshi 
Committed: Tue Sep 27 14:46:35 2016 +0800

--
 examples/test_case_data/sandbox/hbase-site.xml  |  19 +-
 .../test_case_data/sandbox/kylin_job_conf.xml   |  86 ++---
 examples/test_case_data/sandbox/mapred-site.xml |  23 +-
 .../kylin/provision/BuildCubeWithEngine.java|  53 +--
 pom.xml |  20 +-
 .../kylin/rest/security/AclHBaseStorage.java|   4 +-
 .../rest/security/MockAclHBaseStorage.java  |   8 +-
 .../apache/kylin/rest/security/MockHTable.java  |  95 +
 .../rest/security/RealAclHBaseStorage.java  |   9 +-
 .../apache/kylin/rest/service/AclService.java   |  25 +-
 .../apache/kylin/rest/service/CubeService.java  |  36 +-
 .../apache/kylin/rest/service/QueryService.java |  24 +-
 .../apache/kylin/rest/service/UserService.java  |  17 +-
 .../kylin/storage/hbase/HBaseConnection.java| 100 ++---
 .../kylin/storage/hbase/HBaseResourceStore.java |  31 +-
 .../storage/hbase/cube/SimpleHBaseStore.java|  20 +-
 .../hbase/cube/v1/CubeSegmentTupleIterator.java |  11 +-
 .../storage/hbase/cube/v1/CubeStorageQuery.java |   4 +-
 .../hbase/cube/v1/RegionScannerAdapter.java |  10 +-
 .../cube/v1/SerializedHBaseTupleIterator.java   |   4 +-
 .../observer/AggregateRegionObserver.java   |   4 +-
 .../observer/AggregationScanner.java|  14 +-
 .../observer/ObserverAggregationCache.java  |  10 +-
 .../coprocessor/observer/ObserverEnabler.java   |   4 +-
 .../hbase/cube/v2/CubeHBaseEndpointRPC.java |  10 +-
 .../storage/hbase/cube/v2/CubeHBaseScanRPC.java |   9 +-
 .../coprocessor/endpoint/CubeVisitService.java  |   4 +-
 .../storage/hbase/steps/CubeHTableUtil.java |  16 +-
 .../storage/hbase/steps/DeprecatedGCStep.java   |  23 +-
 .../storage/hbase/steps/HBaseCuboidWriter.java  |   8 +-
 .../hbase/steps/HBaseStreamingOutput.java   |   9 +-
 .../kylin/storage/hbase/steps/MergeGCStep.java  |  23 +-
 .../storage/hbase/util/CleanHtableCLI.java  |  12 +-
 .../storage/hbase/util/CubeMigrationCLI.java| 372 +--
 .../hbase/util/CubeMigrationCheckCLI.java   |  19 +-
 .../hbase/util/DeployCoprocessorCLI.java|  22 +-
 .../hbase/util/ExtendCubeToHybridCLI.java   |   8 +-
 .../hbase/util/GridTableHBaseBenchmark.java |  34 +-
 .../kylin/storage/hbase/util/HBaseClean.java|  18 +-
 .../hbase/util/HBaseRegionSizeCalculator.java   |  35 +-
 .../kylin/storage/hbase/util/HBaseUsage.java|   9 +-
 .../storage/hbase/util/HbaseStreamingInput.java |  30 +-
 .../hbase/util/HtableAlterMetadataCLI.java  |   9 +-
 .../storage/hbase/util/OrphanHBaseCleanJob.java |  17 +-
 .../kylin/storage/hbase/util/PingHBaseCLI.java  |  15 +-
 .../kylin/storage/hbase/util/RowCounterCLI.java |  11 +-
 .../storage/hbase/util/StorageCleanupJob.java   |  20 +-
 .../storage/hbase/util/UpdateHTableHostCLI.java |  17 +-
 .../observer/AggregateRegionObserverTest.java   |  31 +-
 .../v1/filter/TestFuzzyRowFilterV2EndToEnd.java |   5 +-
 50 files changed, 657 insertions(+), 760 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/c4e31c1b/examples/test_case_data/sandbox/hbase-site.xml
--
diff --git a/examples/test_case_data/sandbox/hbase-site.xml 
b/examples/test_case_data/sandbox/hbase-site.xml
index 46d5345..734908e 100644
--- a/examples/test_case_data/sandbox/hbase-site.xml
+++ b/examples/test_case_data/sandbox/hbase-site.xml
@@ -190,22 +190,5 @@
 zookeeper.znode.parent
 /hbase-unsecure
 
-
-hbase.client.pause
-100
-General client pause value.  Used mostly as value to wait
-before running a retry of a failed get, region lookup, etc.
-See hbase.client.retries.number for description of how we backoff 
from
-this initial pause amount and how this pause works w/ 
retries.
-
-
-hbase.client.retries.number
-5
-Maximum retries.  Used as maximum for all retryable
-operations such as the getting of a cell's value, starting a row 
update,
-etc.  Retry interval is a rough function based on 
hbase.client.pause.  At
-first we retry at this interval but then with backoff, we pretty 
quickly reach
-retrying every ten 

[10/10] kylin git commit: remove fitler hive dependency logic in AbstractHadoopJob

2016-10-14 Thread mahongbin
remove fitler hive dependency logic in AbstractHadoopJob


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

Branch: refs/heads/1.5.4.1-beeline
Commit: 8f4ed7279173e1e28c51039b8ba19f4e493cbaf8
Parents: 13b7b54
Author: Hongbin Ma 
Authored: Thu Oct 13 21:35:49 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Oct 13 21:35:49 2016 +0800

--
 .../java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/8f4ed727/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
--
diff --git 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
index af2ed9f..8e5e7da 100644
--- 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
+++ 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
@@ -199,7 +199,8 @@ public abstract class AbstractHadoopJob extends Configured 
implements Tool {
 kylinHiveDependency = kylinHiveDependency.replace(":", ",");
 
 logger.info("Hive Dependencies Before Filtered: " + 
kylinHiveDependency);
-String filteredHive = 
filterKylinHiveDependency(kylinHiveDependency);
+//String filteredHive = 
filterKylinHiveDependency(kylinHiveDependency);
+String filteredHive = kylinHiveDependency;
 logger.info("Hive Dependencies After Filtered: " + filteredHive);
 
 if (kylinDependency.length() > 0)



[08/10] kylin git commit: KYLIN-1672 support kylin on cdh 5.7

2016-10-14 Thread mahongbin
KYLIN-1672 support kylin on cdh 5.7


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

Branch: refs/heads/1.5.4.1-beeline
Commit: 3bd0c10fe363fd309377accd73e584cea1d0fdba
Parents: c4e31c1
Author: Lynne Jiang 
Authored: Mon May 16 03:33:27 2016 -0700
Committer: shaofengshi 
Committed: Tue Sep 27 15:30:05 2016 +0800

--
 dev-support/test_all_against_hdp_2_2_4_2_2.sh   |   0
 .../kylin/engine/mr/steps/MockupMapContext.java |  15 +-
 examples/test_case_data/sandbox/core-site.xml   | 146 +++---
 examples/test_case_data/sandbox/hbase-site.xml  | 162 ++
 examples/test_case_data/sandbox/hdfs-site.xml   | 259 ++
 examples/test_case_data/sandbox/mapred-site.xml | 396 ++-
 examples/test_case_data/sandbox/yarn-site.xml   | 496 ++-
 pom.xml |  23 +-
 server/pom.xml  |  36 ++
 .../storage/hbase/steps/MockupMapContext.java   |  19 +-
 tool/pom.xml|   8 +
 11 files changed, 431 insertions(+), 1129 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/3bd0c10f/dev-support/test_all_against_hdp_2_2_4_2_2.sh
--
diff --git a/dev-support/test_all_against_hdp_2_2_4_2_2.sh 
b/dev-support/test_all_against_hdp_2_2_4_2_2.sh
old mode 100644
new mode 100755

http://git-wip-us.apache.org/repos/asf/kylin/blob/3bd0c10f/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java
--
diff --git 
a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java
 
b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java
index 847071d..9900465 100644
--- 
a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java
+++ 
b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/MockupMapContext.java
@@ -77,6 +77,7 @@ public class MockupMapContext {
 outKV[0] = key;
 outKV[1] = value;
 }
+
 }
 
 @Override
@@ -99,6 +100,7 @@ public class MockupMapContext {
 throw new NotImplementedException();
 }
 
+
 @Override
 public float getProgress() {
 throw new NotImplementedException();
@@ -195,17 +197,17 @@ public class MockupMapContext {
 }
 
 @Override
-public RawComparator getSortComparator() {
+public boolean userClassesTakesPrecedence() {
 throw new NotImplementedException();
 }
 
 @Override
-public String getJar() {
+public RawComparator getSortComparator() {
 throw new NotImplementedException();
 }
 
 @Override
-public RawComparator getGroupingComparator() {
+public String getJar() {
 throw new NotImplementedException();
 }
 
@@ -221,7 +223,7 @@ public class MockupMapContext {
 
 @Override
 public boolean getProfileEnabled() {
-throw new NotImplementedException();
+return false;
 }
 
 @Override
@@ -308,6 +310,11 @@ public class MockupMapContext {
 public RawComparator getCombinerKeyGroupingComparator() {
 throw new NotImplementedException();
 }
+
+@Override
+public RawComparator getGroupingComparator() {
+return null;
+}
 });
 }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/3bd0c10f/examples/test_case_data/sandbox/core-site.xml
--
diff --git a/examples/test_case_data/sandbox/core-site.xml 
b/examples/test_case_data/sandbox/core-site.xml
index 9aa588c..6162406 100644
--- a/examples/test_case_data/sandbox/core-site.xml
+++ b/examples/test_case_data/sandbox/core-site.xml
@@ -14,152 +14,146 @@
   See the License for the specific language governing permissions and
   limitations under the License.
 -->
+
 
-
 
 fs.defaultFS
-hdfs://sandbox.hortonworks.com:8020
-true
+hdfs://quickstart.cloudera:8020
 
-
 
 fs.trash.interval
-360
+1
 
-
 
-
ha.failover-controller.active-standby-elector.zk.op.retries
-120
+io.compression.codecs
+

[04/10] kylin git commit: [maven-release-plugin] prepare release kylin-1.5.4.1

2016-10-14 Thread mahongbin
[maven-release-plugin] prepare release kylin-1.5.4.1


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

Branch: refs/heads/1.5.4.1-beeline
Commit: b08871e7dd4595474832b5b6a2e156935b2d73c6
Parents: 2141b21
Author: shaofengshi 
Authored: Sat Sep 24 00:01:07 2016 +
Committer: shaofengshi 
Committed: Sat Sep 24 00:01:07 2016 +

--
 assembly/pom.xml |  5 ++---
 atopcalcite/pom.xml  |  5 ++---
 core-common/pom.xml  |  5 ++---
 core-cube/pom.xml|  5 ++---
 core-dictionary/pom.xml  |  5 ++---
 core-job/pom.xml |  5 ++---
 core-metadata/pom.xml|  5 ++---
 core-storage/pom.xml |  5 ++---
 engine-mr/pom.xml|  5 ++---
 engine-spark/pom.xml |  5 ++---
 engine-streaming/pom.xml |  5 ++---
 jdbc/pom.xml |  5 ++---
 kylin-it/pom.xml | 13 ++---
 pom.xml  | 11 +--
 query/pom.xml|  5 ++---
 server-base/pom.xml  |  5 ++---
 server/pom.xml   |  5 ++---
 source-hive/pom.xml  |  5 ++---
 source-kafka/pom.xml |  5 ++---
 storage-hbase/pom.xml|  8 +++-
 tomcat-ext/pom.xml   |  6 ++
 tool/pom.xml |  5 ++---
 22 files changed, 52 insertions(+), 76 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/b08871e7/assembly/pom.xml
--
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 151ab0d..9bb8477 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -17,12 +17,11 @@
  limitations under the License.
 -->
 
-http://maven.apache.org/POM/4.0.0; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance;
- xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd;>
+http://maven.apache.org/POM/4.0.0; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd;>
 
 kylin
 org.apache.kylin
-1.5.4.1-SNAPSHOT
+1.5.4.1
 
 
 4.0.0

http://git-wip-us.apache.org/repos/asf/kylin/blob/b08871e7/atopcalcite/pom.xml
--
diff --git a/atopcalcite/pom.xml b/atopcalcite/pom.xml
index 744dbb0..50780f2 100644
--- a/atopcalcite/pom.xml
+++ b/atopcalcite/pom.xml
@@ -17,8 +17,7 @@
  limitations under the License.
 -->
 
-http://maven.apache.org/POM/4.0.0; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance;
- xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd;>
+http://maven.apache.org/POM/4.0.0; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd;>
 4.0.0
 
 atopcalcite
@@ -28,7 +27,7 @@
 
 org.apache.kylin
 kylin
-1.5.4.1-SNAPSHOT
+1.5.4.1
 
 
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/b08871e7/core-common/pom.xml
--
diff --git a/core-common/pom.xml b/core-common/pom.xml
index 641f88d..32c9b67 100644
--- a/core-common/pom.xml
+++ b/core-common/pom.xml
@@ -17,8 +17,7 @@
  limitations under the License.
 -->
 
-http://maven.apache.org/POM/4.0.0; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance;
- xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd;>
+http://maven.apache.org/POM/4.0.0; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd;>
 4.0.0
 
 kylin-core-common
@@ -28,7 +27,7 @@
 
 org.apache.kylin
 kylin
-1.5.4.1-SNAPSHOT
+1.5.4.1
 
 
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/b08871e7/core-cube/pom.xml
--
diff --git a/core-cube/pom.xml b/core-cube/pom.xml
index 54f4786..d863b1d 100644
--- a/core-cube/pom.xml
+++ b/core-cube/pom.xml
@@ -17,8 +17,7 @@
  limitations under the License.
 -->
 
-http://maven.apache.org/POM/4.0.0; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance;
- xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd;>
+http://maven.apache.org/POM/4.0.0; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd;>
 4.0.0
 
 

[03/10] kylin git commit: add .gitconfig to rat exclude list

2016-10-14 Thread mahongbin
add .gitconfig to rat exclude list


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

Branch: refs/heads/1.5.4.1-beeline
Commit: 2141b214c1659d005f23b3587de6587d51eec1ff
Parents: d33ba6a
Author: shaofengshi 
Authored: Fri Sep 23 23:56:02 2016 +
Committer: shaofengshi 
Committed: Fri Sep 23 23:56:02 2016 +

--
 pom.xml | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/2141b214/pom.xml
--
diff --git a/pom.xml b/pom.xml
index fc5b511..d2bd442 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1131,6 +1131,7 @@
 **/LICENSE*
 
 .git/**
+.gitconfig
 
 .idea/**
 **/*.iml
@@ -1272,4 +1273,4 @@
 
 
 
-
\ No newline at end of file
+



[01/10] kylin git commit: update version

2016-10-14 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/1.5.4.1-beeline [created] 8f4ed7279


update version


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

Branch: refs/heads/1.5.4.1-beeline
Commit: 18a14547cb923d6c051f48a774cb8b804ad2d510
Parents: 4c9b67f
Author: shaofengshi 
Authored: Fri Sep 23 16:57:26 2016 +0800
Committer: shaofengshi 
Committed: Fri Sep 23 16:57:26 2016 +0800

--
 assembly/pom.xml | 2 +-
 atopcalcite/pom.xml  | 2 +-
 core-common/pom.xml  | 2 +-
 core-cube/pom.xml| 2 +-
 core-dictionary/pom.xml  | 2 +-
 core-job/pom.xml | 2 +-
 core-metadata/pom.xml| 2 +-
 core-storage/pom.xml | 2 +-
 engine-mr/pom.xml| 2 +-
 engine-spark/pom.xml | 2 +-
 engine-streaming/pom.xml | 2 +-
 jdbc/pom.xml | 2 +-
 kylin-it/pom.xml | 2 +-
 pom.xml  | 2 +-
 query/pom.xml| 2 +-
 server-base/pom.xml  | 2 +-
 server/pom.xml   | 2 +-
 source-hive/pom.xml  | 2 +-
 source-kafka/pom.xml | 2 +-
 storage-hbase/pom.xml| 2 +-
 tomcat-ext/pom.xml   | 2 +-
 tool/pom.xml | 2 +-
 22 files changed, 22 insertions(+), 22 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/18a14547/assembly/pom.xml
--
diff --git a/assembly/pom.xml b/assembly/pom.xml
index c7bde96..151ab0d 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -22,7 +22,7 @@
 
 kylin
 org.apache.kylin
-1.6.0-SNAPSHOT
+1.5.4.1-SNAPSHOT
 
 
 4.0.0

http://git-wip-us.apache.org/repos/asf/kylin/blob/18a14547/atopcalcite/pom.xml
--
diff --git a/atopcalcite/pom.xml b/atopcalcite/pom.xml
index 177b933..744dbb0 100644
--- a/atopcalcite/pom.xml
+++ b/atopcalcite/pom.xml
@@ -28,7 +28,7 @@
 
 org.apache.kylin
 kylin
-1.6.0-SNAPSHOT
+1.5.4.1-SNAPSHOT
 
 
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/18a14547/core-common/pom.xml
--
diff --git a/core-common/pom.xml b/core-common/pom.xml
index 780fd82..641f88d 100644
--- a/core-common/pom.xml
+++ b/core-common/pom.xml
@@ -28,7 +28,7 @@
 
 org.apache.kylin
 kylin
-1.6.0-SNAPSHOT
+1.5.4.1-SNAPSHOT
 
 
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/18a14547/core-cube/pom.xml
--
diff --git a/core-cube/pom.xml b/core-cube/pom.xml
index 7c9a549..54f4786 100644
--- a/core-cube/pom.xml
+++ b/core-cube/pom.xml
@@ -28,7 +28,7 @@
 
 org.apache.kylin
 kylin
-1.6.0-SNAPSHOT
+1.5.4.1-SNAPSHOT
 
 
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/18a14547/core-dictionary/pom.xml
--
diff --git a/core-dictionary/pom.xml b/core-dictionary/pom.xml
index 1bf97af..d4940b9 100644
--- a/core-dictionary/pom.xml
+++ b/core-dictionary/pom.xml
@@ -28,7 +28,7 @@
 
 org.apache.kylin
 kylin
-1.6.0-SNAPSHOT
+1.5.4.1-SNAPSHOT
 
 
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/18a14547/core-job/pom.xml
--
diff --git a/core-job/pom.xml b/core-job/pom.xml
index 601f730..a250b07 100644
--- a/core-job/pom.xml
+++ b/core-job/pom.xml
@@ -28,7 +28,7 @@
 
 org.apache.kylin
 kylin
-1.6.0-SNAPSHOT
+1.5.4.1-SNAPSHOT
 
 
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/18a14547/core-metadata/pom.xml
--
diff --git a/core-metadata/pom.xml b/core-metadata/pom.xml
index 142dd33..6948eba 100644
--- a/core-metadata/pom.xml
+++ b/core-metadata/pom.xml
@@ -28,7 +28,7 @@
 
 org.apache.kylin
 kylin
-1.6.0-SNAPSHOT
+1.5.4.1-SNAPSHOT
 
 
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/18a14547/core-storage/pom.xml
--
diff --git a/core-storage/pom.xml b/core-storage/pom.xml
index 2d9d618..e801c9d 100644
--- a/core-storage/pom.xml
+++ b/core-storage/pom.xml
@@ -28,7 +28,7 @@
 
 org.apache.kylin
 kylin
-1.6.0-SNAPSHOT
+1.5.4.1-SNAPSHOT
 
 
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/18a14547/engine-mr/pom.xml

[1/2] kylin git commit: KYLIN-2030 bug fix

2016-10-09 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/orderedbytes adf1369af -> afdec89fe


KYLIN-2030 bug fix


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

Branch: refs/heads/orderedbytes
Commit: b0aa327d23d635fada35b80bb149a2611ad689b5
Parents: adf1369
Author: Hongbin Ma 
Authored: Sun Oct 9 19:07:52 2016 +0800
Committer: Hongbin Ma 
Committed: Sun Oct 9 19:07:52 2016 +0800

--
 .../kylin/cube/CubeCapabilityChecker.java   | 23 ++--
 .../kylin/query/relnode/OLAPAggregateRel.java   | 22 +++
 2 files changed, 25 insertions(+), 20 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/b0aa327d/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
--
diff --git 
a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java 
b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
index ee21b1c..e509d98 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
@@ -81,7 +81,7 @@ public class CubeCapabilityChecker {
 //1. dimension as measure
 
 if (!unmatchedAggregations.isEmpty()) {
-tryDimensionAsMeasures(unmatchedAggregations, digest, cube, 
result, cube.getDescriptor().listDimensionColumnsIncludingDerived());
+tryDimensionAsMeasures(unmatchedAggregations, result, 
cube.getDescriptor().listDimensionColumnsIncludingDerived());
 }
 } else {
 //for non query-on-facttable 
@@ -92,10 +92,18 @@ public class CubeCapabilityChecker {
 dimCols.add(columnDesc.getRef());
 }
 
-//1. dimension as measure, like max(cal_dt) or count( distinct 
col) from lookup
+//1. all aggregations on lookup table can be done. For 
distinct count, mark them all DimensionAsMeasures
+// so that the measure has a chance to be upgraded to 
DimCountDistinctMeasureType in 
org.apache.kylin.metadata.model.FunctionDesc#reInitMeasureType
 if (!unmatchedAggregations.isEmpty()) {
-tryDimensionAsMeasures(unmatchedAggregations, digest, 
cube, result, dimCols);
+Iterator itr = 
unmatchedAggregations.iterator();
+while (itr.hasNext()) {
+FunctionDesc functionDesc = itr.next();
+if 
(dimCols.containsAll(functionDesc.getParameter().getColRefs())) {
+itr.remove();
+}
+}
 }
+tryDimensionAsMeasures(Lists.newArrayList(aggrFunctions), 
result, dimCols);
 
 //2. more "dimensions" contributed by snapshot
 if (!unmatchedDimensions.isEmpty()) {
@@ -159,19 +167,12 @@ public class CubeCapabilityChecker {
 return result;
 }
 
-private static void tryDimensionAsMeasures(Collection 
unmatchedAggregations, SQLDigest digest, CubeInstance cube, CapabilityResult 
result, Set dimCols) {
-CubeDesc cubeDesc = cube.getDescriptor();
-Collection cubeFuncs = cubeDesc.listAllFunctions();
+private static void tryDimensionAsMeasures(Collection 
unmatchedAggregations, CapabilityResult result, Set dimCols) {
 
 Iterator it = unmatchedAggregations.iterator();
 while (it.hasNext()) {
 FunctionDesc functionDesc = it.next();
 
-if (cubeFuncs.contains(functionDesc)) {
-it.remove();
-continue;
-}
-
 // let calcite handle count
 if (functionDesc.isCount()) {
 it.remove();

http://git-wip-us.apache.org/repos/asf/kylin/blob/b0aa327d/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
--
diff --git 
a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java 
b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
index 97efb27..c7a1eff 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
@@ -285,16 +285,20 @@ public class OLAPAggregateRel extends Aggregate 
implements OLAPRel {
 }
 
 private void translateAggregation() {
-// now the realization is known, replace aggregations with what's 
defined on MeasureDesc
- 

[2/2] kylin git commit: KYLIN-1726 fix 'FileSystem Closed' error

2016-10-09 Thread mahongbin
KYLIN-1726 fix 'FileSystem Closed' error


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

Branch: refs/heads/orderedbytes
Commit: afdec89fe09dcb28b368775f8b830c78f74e7489
Parents: b0aa327
Author: shaofengshi 
Authored: Sun Oct 9 19:06:07 2016 +0800
Committer: Hongbin Ma 
Committed: Sun Oct 9 19:08:41 2016 +0800

--
 .../apache/kylin/source/kafka/UpdateTimeRangeStep.java   | 11 ++-
 1 file changed, 10 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/afdec89f/source-kafka/src/main/java/org/apache/kylin/source/kafka/UpdateTimeRangeStep.java
--
diff --git 
a/source-kafka/src/main/java/org/apache/kylin/source/kafka/UpdateTimeRangeStep.java
 
b/source-kafka/src/main/java/org/apache/kylin/source/kafka/UpdateTimeRangeStep.java
index bb64bf9..9e902d8 100644
--- 
a/source-kafka/src/main/java/org/apache/kylin/source/kafka/UpdateTimeRangeStep.java
+++ 
b/source-kafka/src/main/java/org/apache/kylin/source/kafka/UpdateTimeRangeStep.java
@@ -21,6 +21,7 @@ import java.io.BufferedReader;
 import java.io.IOException;
 import java.io.InputStreamReader;
 
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.time.FastDateFormat;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -63,7 +64,12 @@ public class UpdateTimeRangeStep extends AbstractExecutable {
 final Path outputFile = new Path(outputPath, partitionCol.getName());
 
 String minValue = null, maxValue = null, currentValue = null;
-try (FileSystem fs = HadoopUtil.getFileSystem(outputPath); 
FSDataInputStream inputStream = fs.open(outputFile); BufferedReader 
bufferedReader = new BufferedReader(new InputStreamReader(inputStream))) {
+FSDataInputStream inputStream = null;
+BufferedReader bufferedReader = null;
+try {
+FileSystem fs = HadoopUtil.getFileSystem(outputPath);
+inputStream = fs.open(outputFile);
+bufferedReader = new BufferedReader(new 
InputStreamReader(inputStream));
 minValue = currentValue = bufferedReader.readLine();
 while (currentValue != null) {
 maxValue = currentValue;
@@ -72,6 +78,9 @@ public class UpdateTimeRangeStep extends AbstractExecutable {
 } catch (IOException e) {
 logger.error("fail to read file " + outputFile, e);
 return new ExecuteResult(ExecuteResult.State.ERROR, 
e.getLocalizedMessage());
+} finally {
+IOUtils.closeQuietly(bufferedReader);
+IOUtils.closeQuietly(inputStream);
 }
 
 final DataType partitionColType = partitionCol.getType();



kylin git commit: KYLIN-2030 enhancement

2016-10-08 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/orderedbytes 164288ccb -> adf1369af


KYLIN-2030 enhancement


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

Branch: refs/heads/orderedbytes
Commit: adf1369affd9615039cb5a5503fad092b60bf375
Parents: 164288c
Author: Hongbin Ma 
Authored: Sat Oct 8 16:14:26 2016 +0800
Committer: Hongbin Ma 
Committed: Sat Oct 8 16:14:26 2016 +0800

--
 .../kylin/cube/CubeCapabilityChecker.java   | 38 +---
 .../org/apache/kylin/cube/model/CubeDesc.java   | 10 +++---
 .../kylin/metadata/model/FunctionDesc.java  |  1 +
 .../test/resources/query/sql_lookup/query06.sql |  1 +
 .../test/resources/query/sql_lookup/query07.sql |  1 +
 5 files changed, 32 insertions(+), 19 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/adf1369a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
--
diff --git 
a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java 
b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
index 1eada16..ee21b1c 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
@@ -75,22 +75,34 @@ public class CubeCapabilityChecker {
 tryCustomMeasureTypes(unmatchedDimensions, unmatchedAggregations, 
digest, cube, result);
 //}
 
-// try dimension-as-measure
-if (!unmatchedAggregations.isEmpty()) {
-if (cube.getDescriptor().getFactTable().equals(digest.factTable)) {
+//more tricks
+if (cube.getDescriptor().getFactTable().equals(digest.factTable)) {
+//for query-on-facttable
+//1. dimension as measure
+
+if (!unmatchedAggregations.isEmpty()) {
 tryDimensionAsMeasures(unmatchedAggregations, digest, cube, 
result, cube.getDescriptor().listDimensionColumnsIncludingDerived());
-} else {
-//deal with query on lookup table, like 
https://issues.apache.org/jira/browse/KYLIN-2030
-if 
(cube.getSegments().get(0).getSnapshots().containsKey(digest.factTable)) {
-TableDesc tableDesc = 
MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getTableDesc(digest.factTable);
-Set dimCols = Sets.newHashSet();
-for (ColumnDesc columnDesc : tableDesc.getColumns()) {
-dimCols.add(columnDesc.getRef());
-}
+}
+} else {
+//for non query-on-facttable 
+if 
(cube.getSegments().get(0).getSnapshots().containsKey(digest.factTable)) {
+TableDesc tableDesc = 
MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getTableDesc(digest.factTable);
+Set dimCols = Sets.newHashSet();
+for (ColumnDesc columnDesc : tableDesc.getColumns()) {
+dimCols.add(columnDesc.getRef());
+}
+
+//1. dimension as measure, like max(cal_dt) or count( distinct 
col) from lookup
+if (!unmatchedAggregations.isEmpty()) {
 tryDimensionAsMeasures(unmatchedAggregations, digest, 
cube, result, dimCols);
-} else {
-logger.info("Skip tryDimensionAsMeasures because current 
cube {} does not touch lookup table {} at all", cube.getName(), 
digest.factTable);
 }
+
+//2. more "dimensions" contributed by snapshot
+if (!unmatchedDimensions.isEmpty()) {
+unmatchedDimensions.removeAll(dimCols);
+}
+} else {
+logger.info("cube {} does not touch lookup table {} at all", 
cube.getName(), digest.factTable);
 }
 }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/adf1369a/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 4195451..7e055be 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
@@ -33,9 +33,9 @@ import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;

kylin git commit: KYLIN-1726 fix BuildCubeWithStream

2016-10-08 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/orderedbytes 5064b6b1a -> 164288ccb


KYLIN-1726 fix BuildCubeWithStream


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

Branch: refs/heads/orderedbytes
Commit: 164288ccb1c7ff529703402f687181d818b2c3fc
Parents: 5064b6b
Author: Hongbin Ma 
Authored: Sat Oct 8 14:08:41 2016 +0800
Committer: Hongbin Ma 
Committed: Sat Oct 8 14:08:41 2016 +0800

--
 .../kylin/provision/BuildCubeWithStream.java| 27 +++-
 1 file changed, 9 insertions(+), 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/164288cc/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
--
diff --git 
a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java 
b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
index f8805a6..d431005 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
@@ -18,6 +18,8 @@
 
 package org.apache.kylin.provision;
 
+import static java.lang.Thread.sleep;
+
 import java.io.File;
 import java.io.IOException;
 import java.text.ParseException;
@@ -32,10 +34,8 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.FutureTask;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.collect.Lists;
 import org.I0Itec.zkclient.ZkConnection;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.hadoop.util.ToolRunner;
 import org.apache.kafka.common.requests.MetadataResponse;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.ClassUtil;
@@ -65,7 +65,7 @@ import org.junit.Assert;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static java.lang.Thread.sleep;
+import com.google.common.collect.Lists;
 
 /**
  *  for streaming cubing case "test_streaming_table"
@@ -205,7 +205,7 @@ public class BuildCubeWithStream {
 for (int i = 0; i < futures.size(); i++) {
 ExecutableState result = futures.get(i).get(20, TimeUnit.MINUTES);
 logger.info("Checking building task " + i + " whose state is " + 
result);
-Assert.assertTrue(result == null || result == 
ExecutableState.SUCCEED || result == ExecutableState.DISCARDED );
+Assert.assertTrue(result == null || result == 
ExecutableState.SUCCEED || result == ExecutableState.DISCARDED);
 if (result == ExecutableState.SUCCEED)
 succeedBuild++;
 }
@@ -214,7 +214,6 @@ public class BuildCubeWithStream {
 List segments = 
cubeManager.getCube(cubeName).getSegments(SegmentStatusEnum.READY);
 Assert.assertTrue(segments.size() == succeedBuild);
 
-
 if (fastBuildMode == false) {
 //empty build
 ExecutableState result = buildSegment(cubeName, 0, Long.MAX_VALUE);
@@ -238,7 +237,6 @@ public class BuildCubeWithStream {
 logger.info("Build is done");
 }
 
-
 private ExecutableState mergeSegment(String cubeName, long startOffset, 
long endOffset) throws Exception {
 CubeSegment segment = 
cubeManager.mergeSegments(cubeManager.getCube(cubeName), 0, 0, startOffset, 
endOffset, false);
 DefaultChainedExecutable job = 
EngineFactory.createBatchMergeJob(segment, "TEST");
@@ -279,14 +277,12 @@ public class BuildCubeWithStream {
 
HBaseMetadataTestCase.staticCreateTestMetadata(HBaseMetadataTestCase.SANDBOX_TEST_DATA);
 }
 
-public static void afterClass() throws Exception {
-cleanupOldStorage();
-HBaseMetadataTestCase.staticCleanupTestMetadata();
-}
-
 public void after() {
 kafkaServer.stop();
 DefaultScheduler.destroyInstance();
+
+cleanupOldStorage();
+HBaseMetadataTestCase.staticCleanupTestMetadata();
 }
 
 protected void waitForJob(String jobId) {
@@ -304,7 +300,7 @@ public class BuildCubeWithStream {
 }
 }
 
-private static void cleanupOldStorage() throws Exception {
+protected void cleanupOldStorage() {
 String[] args = { "--delete", "true" };
 StorageCleanupJob cli = new StorageCleanupJob();
 cli.execute(args);
@@ -317,17 +313,12 @@ public class BuildCubeWithStream {
 buildCubeWithStream = new BuildCubeWithStream();
 buildCubeWithStream.before();
 buildCubeWithStream.build();
+buildCubeWithStream.after();
 logger.info("Going to exit");
 

[28/50] [abbrv] kylin git commit: KYLIN-1962 reorg BuildCubeWithStream test case

2016-10-07 Thread mahongbin
KYLIN-1962 reorg BuildCubeWithStream test case

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

Branch: refs/heads/orderedbytes
Commit: 859230d787f9c218f56e56308897b68fb23d8dc4
Parents: ab5563a
Author: shaofengshi 
Authored: Mon Sep 26 18:10:32 2016 +0800
Committer: shaofengshi 
Committed: Tue Sep 27 10:17:40 2016 +0800

--
 .../apache/kylin/common/KylinConfigBase.java|   2 +-
 .../java/org/apache/kylin/cube/CubeManager.java |  20 +++
 .../java/org/apache/kylin/cube/CubeSegment.java |   7 +-
 .../test_streaming_table_cube_desc.json |  17 ++-
 .../test_streaming_table_model_desc.json|   3 +-
 .../kylin/provision/BuildCubeWithStream.java| 121 +++--
 .../kylin/provision/BuildCubeWithStream2.java   | 134 ---
 7 files changed, 116 insertions(+), 188 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/859230d7/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
--
diff --git 
a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java 
b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index 3e41055..838ef97 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -807,6 +807,6 @@ abstract public class KylinConfigBase implements 
Serializable {
 }
 
 public int getMaxBuildingSegments() {
-return Integer.parseInt(getOptional("kylin.cube.building.segment.max", 
"1"));
+return Integer.parseInt(getOptional("kylin.cube.building.segment.max", 
"2"));
 }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/859230d7/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 463c8e9..962568c 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
@@ -29,10 +29,12 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 import java.util.Random;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 
+import com.google.common.collect.Maps;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.KylinConfigExt;
@@ -476,6 +478,24 @@ public class CubeManager implements IRealizationProvider {
 if (pair.getFirst() == false || pair.getSecond() == false)
 throw new IllegalArgumentException("The new refreshing segment " + 
newSegment + " does not match any existing segment in cube " + cube);
 
+if (startOffset > 0 || endOffset > 0) {
+CubeSegment toRefreshSeg = null;
+for (CubeSegment cubeSegment : cube.getSegments()) {
+if (cubeSegment.getSourceOffsetStart() == startOffset && 
cubeSegment.getSourceOffsetEnd() == endOffset) {
+toRefreshSeg = cubeSegment;
+break;
+}
+}
+
+if (toRefreshSeg == null) {
+throw new IllegalArgumentException("For streaming cube, only 
one segment can be refreshed at one time");
+}
+
+Map partitionInfo = Maps.newHashMap();
+partitionInfo.putAll(toRefreshSeg.getAdditionalInfo());
+newSegment.setAdditionalInfo(partitionInfo);
+}
+
 CubeUpdate cubeBuilder = new CubeUpdate(cube);
 cubeBuilder.setToAddSegs(newSegment);
 updateCube(cubeBuilder);

http://git-wip-us.apache.org/repos/asf/kylin/blob/859230d7/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
--
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java 
b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
index afb0d28..d5de47f 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeSegment.java
@@ -20,7 +20,6 @@ package org.apache.kylin.cube;
 
 import java.text.SimpleDateFormat;
 import java.util.Collection;
-import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -106,7 +105,7 @@ 

[42/50] [abbrv] kylin git commit: minor, enhance dev-support scripts

2016-10-07 Thread mahongbin
minor, enhance dev-support scripts


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

Branch: refs/heads/orderedbytes
Commit: 937cc546d1dc34832217f1b806f3e91aa486caa4
Parents: e22039d
Author: Yang Li 
Authored: Tue Oct 4 15:45:45 2016 +0800
Committer: Yang Li 
Committed: Tue Oct 4 15:45:45 2016 +0800

--
 dev-support/sync_hbase_cdh_branches.cmd   | 44 +-
 dev-support/test_all_against_hdp_2_2_4_2_2.sh |  2 +-
 2 files changed, 35 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/937cc546/dev-support/sync_hbase_cdh_branches.cmd
--
diff --git a/dev-support/sync_hbase_cdh_branches.cmd 
b/dev-support/sync_hbase_cdh_branches.cmd
index 0febfbb..ed828ca 100644
--- a/dev-support/sync_hbase_cdh_branches.cmd
+++ b/dev-support/sync_hbase_cdh_branches.cmd
@@ -1,19 +1,43 @@
-git checkout master
-git pull
-git reset apache/master --hard
+#!/bin/bash
+
+# 
+
+base=master
+
+# 
+
+set -o pipefail  # trace ERR through pipes
+set -o errtrace  # trace ERR through 'time command' and other functions
+function error() {
+   SCRIPT="$0"   # script name
+   LASTLINE="$1" # line of error occurrence
+   LASTERR="$2"  # error code
+   echo "ERROR exit from ${SCRIPT} : line ${LASTLINE} with exit code 
${LASTERR}"
+   exit 1
+}
+trap 'error ${LINENO} ${?}' ERR
+
+# 
 
-git checkout apache/1.5.x-HBase1.x
+git fetch apache
+git checkout apache/$base-hbase1.x
 git format-patch -1
-git checkout master
+git checkout apache/$base-cdh1.x
+git format-patch -1
+
+git checkout apache/$base
+git checkout -b tmp
+git reset apache/$base --hard
+
 git am -3 --ignore-whitespace 
0001-KYLIN-1528-Create-a-branch-for-v1.5-with-HBase-1.x-A.patch
-git push apache master:1.5.x-HBase1.x -f
+#git push apache tmp:$base-hbase1.x -f
 rm 0001-KYLIN-1528-Create-a-branch-for-v1.5-with-HBase-1.x-A.patch
 
-git checkout apache/1.5.x-CDH5.7
-git format-patch -1
-git checkout master
 git am -3 --ignore-whitespace 0001-KYLIN-1672-support-kylin-on-cdh-5.7.patch
-git push apache master:1.5.x-CDH5.7 -f
+#git push apache tmp:$base-cdh5.7 -f
 rm 0001-KYLIN-1672-support-kylin-on-cdh-5.7.patch
 
+# clean up
+git checkout master
 git reset apache/master --hard
+git checkout -b tmp

http://git-wip-us.apache.org/repos/asf/kylin/blob/937cc546/dev-support/test_all_against_hdp_2_2_4_2_2.sh
--
diff --git a/dev-support/test_all_against_hdp_2_2_4_2_2.sh 
b/dev-support/test_all_against_hdp_2_2_4_2_2.sh
index f0d27e4..f7780dd 100644
--- a/dev-support/test_all_against_hdp_2_2_4_2_2.sh
+++ b/dev-support/test_all_against_hdp_2_2_4_2_2.sh
@@ -22,4 +22,4 @@ cd ${dir}
 cd ..
 
 mvn clean install -DskipTests 2>&1 | tee mci.log
-mvn verify -Dhdp.version=2.2.4.2-2 -fae 2>&1 | tee mvnverify.log
+mvn verify -Dhdp.version=${HDP_VERSION:-"2.2.4.2-2"} -fae 2>&1 | tee 
mvnverify.log



[08/50] [abbrv] kylin git commit: KYLIN-1565: Read KV max size from HBase config

2016-10-07 Thread mahongbin
KYLIN-1565: Read KV max size from HBase config


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

Branch: refs/heads/orderedbytes
Commit: e05cb60061a43987f7648d3d04921b5a01bb5ac9
Parents: 0861bfb
Author: Yiming Liu 
Authored: Fri Sep 16 11:51:11 2016 +0800
Committer: Yang Li 
Committed: Mon Sep 26 19:45:19 2016 +0800

--
 .../java/org/apache/kylin/common/KylinConfigBase.java |  4 
 .../kylin/storage/hbase/ITHBaseResourceStoreTest.java | 10 +-
 .../apache/kylin/storage/hbase/HBaseResourceStore.java|  2 +-
 3 files changed, 6 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/e05cb600/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
--
diff --git 
a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java 
b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
index 79ee084..ceb188e 100644
--- a/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
+++ b/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java
@@ -570,10 +570,6 @@ abstract public class KylinConfigBase implements 
Serializable {
 return 
getOptionalIntArray("kylin.query.metrics.percentiles.intervals", dft);
 }
 
-public int getHBaseKeyValueSize() {
-return 
Integer.parseInt(this.getOptional("kylin.hbase.client.keyvalue.maxsize", 
"10485760"));
-}
-
 public String getDefaultIGTStorage() {
 return getOptional("kylin.query.storage.default.gtstorage", 
"org.apache.kylin.storage.hbase.cube.v2.CubeHBaseEndpointRPC");
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e05cb600/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
--
diff --git 
a/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
 
b/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
index bc5cdf1..5a2d5dd 100644
--- 
a/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
+++ 
b/kylin-it/src/test/java/org/apache/kylin/storage/hbase/ITHBaseResourceStoreTest.java
@@ -57,12 +57,12 @@ public class ITHBaseResourceStoreTest extends 
HBaseMetadataTestCase {
 String path = "/cube/_test_large_cell.json";
 String largeContent = "THIS_IS_A_LARGE_CELL";
 StringEntity content = new StringEntity(largeContent);
-KylinConfig config = KylinConfig.getInstanceFromEnv();
-int origSize = config.getHBaseKeyValueSize();
+Configuration hconf = HBaseConnection.getCurrentHBaseConfiguration();
+int origSize = 
Integer.parseInt(hconf.get("hbase.client.keyvalue.maxsize", "10485760"));
 ResourceStore store = 
ResourceStore.getStore(KylinConfig.getInstanceFromEnv());
 
 try {
-config.setProperty("kylin.hbase.client.keyvalue.maxsize", 
String.valueOf(largeContent.length() - 1));
+hconf.set("hbase.client.keyvalue.maxsize", 
String.valueOf(largeContent.length() - 1));
 
 store.deleteResource(path);
 
@@ -72,7 +72,7 @@ public class ITHBaseResourceStoreTest extends 
HBaseMetadataTestCase {
 assertEquals(content, t);
 
 Path redirectPath = ((HBaseResourceStore) 
store).bigCellHDFSPath(path);
-Configuration hconf = 
HBaseConnection.getCurrentHBaseConfiguration();
+
 FileSystem fileSystem = FileSystem.get(hconf);
 assertTrue(fileSystem.exists(redirectPath));
 
@@ -82,7 +82,7 @@ public class ITHBaseResourceStoreTest extends 
HBaseMetadataTestCase {
 
 store.deleteResource(path);
 } finally {
-config.setProperty("kylin.hbase.client.keyvalue.maxsize", "" + 
origSize);
+hconf.set("hbase.client.keyvalue.maxsize", "" + origSize);
 store.deleteResource(path);
 }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e05cb600/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
--
diff --git 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
index e2f3661..ee4b7e4 100644
--- 
a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
+++ 
b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/HBaseResourceStore.java
@@ 

[41/50] [abbrv] kylin git commit: minor, clean error handling on Job classes

2016-10-07 Thread mahongbin
minor, clean error handling on Job classes


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

Branch: refs/heads/orderedbytes
Commit: e22039d1344f6bb768df62f73737066a7f10838a
Parents: 92fc0e5
Author: Yang Li 
Authored: Fri Sep 30 22:05:12 2016 +0800
Committer: Yang Li 
Committed: Fri Sep 30 22:05:59 2016 +0800

--
 .../engine/mr/steps/CreateDictionaryJob.java| 34 -
 .../apache/kylin/engine/mr/steps/CuboidJob.java |  4 --
 .../engine/mr/steps/FactDistinctColumnsJob.java |  4 --
 .../kylin/engine/mr/steps/InMemCuboidJob.java   |  4 --
 .../kylin/engine/mr/steps/MergeCuboidJob.java   |  4 --
 .../engine/mr/steps/MetadataCleanupJob.java | 21 +++---
 .../mr/steps/RowKeyDistributionCheckerJob.java  | 59 +++
 .../cardinality/HiveColumnCardinalityJob.java   | 76 +---
 .../kylin/storage/hbase/steps/BulkLoadJob.java  | 54 +++---
 .../storage/hbase/steps/CreateHTableJob.java| 27 +++
 .../kylin/storage/hbase/steps/CubeHFileJob.java |  4 --
 11 files changed, 118 insertions(+), 173 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/e22039d1/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CreateDictionaryJob.java
--
diff --git 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CreateDictionaryJob.java
 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CreateDictionaryJob.java
index 69c0095..5db7c88 100644
--- 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CreateDictionaryJob.java
+++ 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CreateDictionaryJob.java
@@ -40,29 +40,23 @@ public class CreateDictionaryJob extends AbstractHadoopJob {
 @Override
 public int run(String[] args) throws Exception {
 Options options = new Options();
+options.addOption(OPTION_CUBE_NAME);
+options.addOption(OPTION_SEGMENT_ID);
+options.addOption(OPTION_INPUT_PATH);
+parseOptions(options, args);
 
-try {
-options.addOption(OPTION_CUBE_NAME);
-options.addOption(OPTION_SEGMENT_ID);
-options.addOption(OPTION_INPUT_PATH);
-parseOptions(options, args);
+final String cubeName = getOptionValue(OPTION_CUBE_NAME);
+final String segmentID = getOptionValue(OPTION_SEGMENT_ID);
+final String factColumnsInputPath = getOptionValue(OPTION_INPUT_PATH);
 
-final String cubeName = getOptionValue(OPTION_CUBE_NAME);
-final String segmentID = getOptionValue(OPTION_SEGMENT_ID);
-final String factColumnsInputPath = 
getOptionValue(OPTION_INPUT_PATH);
+KylinConfig config = KylinConfig.getInstanceFromEnv();
 
-KylinConfig config = KylinConfig.getInstanceFromEnv();
-
-DictionaryGeneratorCLI.processSegment(config, cubeName, segmentID, 
new DistinctColumnValuesProvider() {
-@Override
-public ReadableTable getDistinctValuesFor(TblColRef col) {
-return new DFSFileTable(factColumnsInputPath + "/" + 
col.getName(), -1);
-}
-});
-} catch (Exception e) {
-printUsage(options);
-throw e;
-}
+DictionaryGeneratorCLI.processSegment(config, cubeName, segmentID, new 
DistinctColumnValuesProvider() {
+@Override
+public ReadableTable getDistinctValuesFor(TblColRef col) {
+return new DFSFileTable(factColumnsInputPath + "/" + 
col.getName(), -1);
+}
+});
 
 return returnCode;
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/e22039d1/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java
--
diff --git 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java
index 0399300..6b0c86e 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidJob.java
@@ -140,10 +140,6 @@ public class CuboidJob extends AbstractHadoopJob {
 this.deletePath(job.getConfiguration(), output);
 
 return waitForCompletion(job);
-} catch (Exception e) {
-logger.error("error in CuboidJob", e);
-printUsage(options);
-throw e;
 } finally {
 if (job != null)
 

[03/50] [abbrv] kylin git commit: KYLIN-2033 Broadcaster stronger sync locking and more comments

2016-10-07 Thread mahongbin
KYLIN-2033 Broadcaster stronger sync locking and more comments


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

Branch: refs/heads/orderedbytes
Commit: e10f2b922006a002ea9cb58ff11a4ecd9aa749c9
Parents: bf127a9
Author: Yang Li 
Authored: Sun Sep 25 13:54:51 2016 +0800
Committer: Yang Li 
Committed: Sun Sep 25 13:54:51 2016 +0800

--
 .../kylin/metadata/cachesync/Broadcaster.java   | 119 ++-
 1 file changed, 65 insertions(+), 54 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/e10f2b92/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
--
diff --git 
a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
 
b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
index 75b2333..8d34cc0 100644
--- 
a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
+++ 
b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
@@ -43,7 +43,16 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 /**
- * Broadcast kylin event out
+ * Broadcast metadata changes across all Kylin servers.
+ * 
+ * The origin server announce the event via Rest API to all Kylin servers 
including itself.
+ * On target server, listeners are registered to process events. As part of 
processing, a 
+ * listener can re-notify a new event to other local listeners.
+ * 
+ * A typical project schema change event:
+ * - model is update on origin server, a "model" update event is announced
+ * - on all servers, model listener is invoked, reload the model, and notify a 
"project_schema" update event
+ * - all listeners respond to the "project_schema" update -- reload cube desc, 
clear project L2 cache, clear calcite data source etc
  */
 public class Broadcaster {
 
@@ -57,13 +66,9 @@ public class Broadcaster {
 private static final ConcurrentHashMap CACHE = 
new ConcurrentHashMap();
 
 public static Broadcaster getInstance(KylinConfig config) {
-Broadcaster r = CACHE.get(config);
-if (r != null) {
-return r;
-}
 
-synchronized (Broadcaster.class) {
-r = CACHE.get(config);
+synchronized (CACHE) {
+Broadcaster r = CACHE.get(config);
 if (r != null) {
 return r;
 }
@@ -79,7 +84,9 @@ public class Broadcaster {
 
 // call Broadcaster.getInstance().notifyClearAll() to clear cache
 static void clearCache() {
-CACHE.clear();
+synchronized (CACHE) {
+CACHE.clear();
+}
 }
 
 // 

@@ -134,22 +141,24 @@ public class Broadcaster {
 }
 
 public void registerListener(Listener listener, String... entities) {
-// ignore re-registration
-List all = listenerMap.get(SYNC_ALL);
-if (all != null && all.contains(listener)) {
-return;
-}
+synchronized (CACHE) {
+// ignore re-registration
+List all = listenerMap.get(SYNC_ALL);
+if (all != null && all.contains(listener)) {
+return;
+}
 
-for (String entity : entities) {
-if (!StringUtils.isBlank(entity))
-addListener(entity, listener);
+for (String entity : entities) {
+if (!StringUtils.isBlank(entity))
+addListener(entity, listener);
+}
+addListener(SYNC_ALL, listener);
+addListener(SYNC_PRJ_SCHEMA, listener);
+addListener(SYNC_PRJ_DATA, listener);
 }
-addListener(SYNC_ALL, listener);
-addListener(SYNC_PRJ_SCHEMA, listener);
-addListener(SYNC_PRJ_DATA, listener);
 }
 
-synchronized private void addListener(String entity, Listener listener) {
+private void addListener(String entity, Listener listener) {
 List list = listenerMap.get(entity);
 if (list == null) {
 list = new ArrayList<>();
@@ -170,42 +179,44 @@ public class Broadcaster {
 notifyListener(SYNC_PRJ_DATA, Event.UPDATE, project);
 }
 
-public synchronized void notifyListener(String entity, Event event, String 
cacheKey) throws IOException {
-List list = listenerMap.get(entity);
-if (list == null)
-return;
-
-logger.debug("Broadcasting metadata 

[37/50] [abbrv] kylin git commit: minor, retry when http failed in job diagnosis

2016-10-07 Thread mahongbin
minor, retry when http failed in job diagnosis


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

Branch: refs/heads/orderedbytes
Commit: 1e049817856ede06c7c8736ad1d608765f301a21
Parents: 51b5b31
Author: lidongsjtu 
Authored: Thu Sep 29 09:48:58 2016 +0800
Committer: lidongsjtu 
Committed: Thu Sep 29 15:34:25 2016 +0800

--
 .../java/org/apache/kylin/tool/JobDiagnosisInfoCLI.java | 4 ++--
 .../java/org/apache/kylin/tool/JobTaskCounterExtractor.java | 9 +++--
 2 files changed, 9 insertions(+), 4 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/1e049817/tool/src/main/java/org/apache/kylin/tool/JobDiagnosisInfoCLI.java
--
diff --git a/tool/src/main/java/org/apache/kylin/tool/JobDiagnosisInfoCLI.java 
b/tool/src/main/java/org/apache/kylin/tool/JobDiagnosisInfoCLI.java
index 635a2c3..c8ff1f4 100644
--- a/tool/src/main/java/org/apache/kylin/tool/JobDiagnosisInfoCLI.java
+++ b/tool/src/main/java/org/apache/kylin/tool/JobDiagnosisInfoCLI.java
@@ -99,8 +99,8 @@ public class JobDiagnosisInfoCLI extends 
AbstractInfoExtractor {
 addRequired(ResourceStore.EXECUTE_RESOURCE_ROOT + "/" + jobId);
 addRequired(ResourceStore.EXECUTE_OUTPUT_RESOURCE_ROOT + "/" + jobId);
 for (ExecutablePO task : executablePO.getTasks()) {
-addRequired(ResourceStore.EXECUTE_RESOURCE_ROOT + "/" + 
executablePO.getTasks());
-addRequired(ResourceStore.EXECUTE_OUTPUT_RESOURCE_ROOT + "/" + 
executablePO.getTasks());
+addRequired(ResourceStore.EXECUTE_RESOURCE_ROOT + "/" + 
task.getUuid());
+addRequired(ResourceStore.EXECUTE_OUTPUT_RESOURCE_ROOT + "/" + 
task.getUuid());
 if (includeYarnLogs) {
 yarnLogsResources.add(task.getUuid());
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/1e049817/tool/src/main/java/org/apache/kylin/tool/JobTaskCounterExtractor.java
--
diff --git 
a/tool/src/main/java/org/apache/kylin/tool/JobTaskCounterExtractor.java 
b/tool/src/main/java/org/apache/kylin/tool/JobTaskCounterExtractor.java
index 7b9d8f3..6a317e9 100644
--- a/tool/src/main/java/org/apache/kylin/tool/JobTaskCounterExtractor.java
+++ b/tool/src/main/java/org/apache/kylin/tool/JobTaskCounterExtractor.java
@@ -47,6 +47,8 @@ public class JobTaskCounterExtractor extends 
AbstractInfoExtractor {
 private String yarnUrl;
 private static final Logger logger = 
LoggerFactory.getLogger(JobTaskCounterExtractor.class);
 
+private final int HTTP_RETRY = 3;
+
 public JobTaskCounterExtractor(String mrJobId) {
 this.mrJobId = mrJobId;
 this.yarnUrl = getRestCheckUrl();
@@ -87,14 +89,17 @@ public class JobTaskCounterExtractor extends 
AbstractInfoExtractor {
 private String getHttpResponse(String url) {
 HttpClient client = new HttpClient();
 String response = null;
-while (response == null) {
+int retry_times = 0;
+while (response == null && retry_times < HTTP_RETRY) {
+retry_times++;
+
 HttpMethod get = new GetMethod(url);
 try {
 get.addRequestHeader("accept", "application/json");
 client.executeMethod(get);
 response = get.getResponseBodyAsString();
 } catch (Exception e) {
-logger.warn("Failed to fetch http response" + e);
+logger.warn("Failed to fetch http response. Retry={}", 
retry_times, e);
 } finally {
 get.releaseConnection();
 }



[04/50] [abbrv] kylin git commit: KYLIN-1978: fix compatible issue on Ubuntu

2016-10-07 Thread mahongbin
KYLIN-1978: fix compatible issue on Ubuntu

Signed-off-by: shaofengshi 


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

Branch: refs/heads/orderedbytes
Commit: a25131eca336c0698a75c85c64dc000c8b91cc74
Parents: e10f2b9
Author: Yiming Liu 
Authored: Sun Sep 18 10:35:17 2016 +0800
Committer: shaofengshi 
Committed: Sun Sep 25 15:38:21 2016 +0800

--
 build/bin/get-properties.sh | 10 ++
 1 file changed, 2 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/a25131ec/build/bin/get-properties.sh
--
diff --git a/build/bin/get-properties.sh b/build/bin/get-properties.sh
index a53de6b..516e8c9 100755
--- a/build/bin/get-properties.sh
+++ b/build/bin/get-properties.sh
@@ -23,11 +23,5 @@ then
 exit -1
 fi
 
-IFS=$'\n'
-result=
-for i in `cat ${KYLIN_HOME}/conf/kylin.properties | grep -w "^$1" | grep -v 
'^#' | awk -F= '{ n = index($0,"="); print substr($0,n+1)}' | cut -c 1-`
-do
-   :
-   result=$i
-done
-echo $result
\ No newline at end of file
+result=`cat ${KYLIN_HOME}/conf/kylin.properties | grep -w "^$1" | grep -v '^#' 
| awk -F= '{ n = index($0,"="); print substr($0,n+1)}' | cut -c 1- |tail -1`
+echo $result



[32/50] [abbrv] kylin git commit: KYLIN-2050: Better close resource for AutoCloseable object

2016-10-07 Thread mahongbin
KYLIN-2050: Better close resource for AutoCloseable object


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

Branch: refs/heads/orderedbytes
Commit: 31b80905ec3645ecf2c0e8bb5c02bdc451974648
Parents: d366dc1
Author: Yiming Liu 
Authored: Sun Sep 25 21:01:33 2016 +0800
Committer: Li Yang 
Committed: Wed Sep 28 13:37:26 2016 +0800

--
 .../org/apache/kylin/common/util/DBUtils.java   | 89 
 .../apache/kylin/common/util/ZipFileUtils.java  |  3 +-
 .../apache/kylin/dict/lookup/LookupTable.java   |  3 +-
 .../apache/kylin/metadata/MetadataManager.java  |  4 +-
 .../java/org/apache/kylin/query/QueryCli.java   | 13 +--
 .../kylin/query/enumerator/HiveEnumerator.java  | 19 +
 .../apache/kylin/rest/service/CubeService.java  |  5 +-
 .../apache/kylin/rest/service/QueryService.java | 23 +
 .../storage/hbase/steps/CubeHTableUtil.java |  7 +-
 .../storage/hbase/steps/DeprecatedGCStep.java   |  8 +-
 .../hbase/util/HBaseRegionSizeCalculator.java   |  3 +-
 .../storage/hbase/util/HbaseStreamingInput.java |  5 +-
 .../kylin/storage/hbase/util/PingHBaseCLI.java  | 13 +--
 13 files changed, 123 insertions(+), 72 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/31b80905/core-common/src/main/java/org/apache/kylin/common/util/DBUtils.java
--
diff --git 
a/core-common/src/main/java/org/apache/kylin/common/util/DBUtils.java 
b/core-common/src/main/java/org/apache/kylin/common/util/DBUtils.java
new file mode 100644
index 000..9ecd3c3
--- /dev/null
+++ b/core-common/src/main/java/org/apache/kylin/common/util/DBUtils.java
@@ -0,0 +1,89 @@
+/*
+ * 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.common.util;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DBUtils {
+
+private static final Logger logger = 
LoggerFactory.getLogger(DBUtils.class);
+
+/**
+ * Closes an ResultSet unconditionally.
+ * 
+ * Equivalent to {@link ResultSet#close()}, except any exceptions will be 
ignored.
+ * This is typically used in finally blocks.
+ * 
+ *
+ * @param output the ResultSet to close, may be null or already closed
+ */
+public static void closeQuietly(final ResultSet rs) {
+closeQuietly((AutoCloseable) rs);
+}
+
+/**
+ * Closes an Statement unconditionally.
+ * 
+ * Equivalent to {@link Statement#close()}, except any exceptions will be 
ignored.
+ * This is typically used in finally blocks.
+ * 
+ *
+ * @param output the ResultSet to close, may be null or already closed
+ */
+public static void closeQuietly(final Statement stmt) {
+closeQuietly((AutoCloseable) stmt);
+}
+
+/**
+ * Closes an Connection unconditionally.
+ * 
+ * Equivalent to {@link Connection#close()}, except any exceptions will be 
ignored.
+ * This is typically used in finally blocks.
+ * 
+ *
+ * @param output the ResultSet to close, may be null or already closed
+ */
+public static void closeQuietly(final Connection conn) {
+closeQuietly((AutoCloseable) conn);
+}
+
+/**
+ * Closes a AutoCloseable unconditionally.
+ * 
+ * Equivalent to {@link AutoCloseable#close()}, except any exceptions will 
be ignored. This is typically used in
+ * finally blocks.
+ * 
+ *
+ * @param closeable the objects to close, may be null or already closed
+ */
+public static void closeQuietly(final AutoCloseable closeable) {
+try {
+if (closeable != null) {
+closeable.close();
+}
+} catch (final Exception ioe) {
+logger.debug("", ioe);

[16/50] [abbrv] kylin git commit: rename license header file

2016-10-07 Thread mahongbin
rename license header file


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

Branch: refs/heads/orderedbytes
Commit: a00d1e37483a5f15bf8f469fb1ed23bba1aa0011
Parents: afd3e61
Author: shaofengshi 
Authored: Tue Sep 27 09:55:03 2016 +0800
Committer: shaofengshi 
Committed: Tue Sep 27 09:55:03 2016 +0800

--
 dev-support/checkstyle-apache-header.txt | 17 +
 dev-support/checkstyle-java-header.txt   | 17 -
 dev-support/checkstyle.xml   |  2 +-
 3 files changed, 18 insertions(+), 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/a00d1e37/dev-support/checkstyle-apache-header.txt
--
diff --git a/dev-support/checkstyle-apache-header.txt 
b/dev-support/checkstyle-apache-header.txt
new file mode 100644
index 000..3740126
--- /dev/null
+++ b/dev-support/checkstyle-apache-header.txt
@@ -0,0 +1,17 @@
+/*
+ * 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.
+*/
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/a00d1e37/dev-support/checkstyle-java-header.txt
--
diff --git a/dev-support/checkstyle-java-header.txt 
b/dev-support/checkstyle-java-header.txt
deleted file mode 100644
index 3740126..000
--- a/dev-support/checkstyle-java-header.txt
+++ /dev/null
@@ -1,17 +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.
-*/
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/a00d1e37/dev-support/checkstyle.xml
--
diff --git a/dev-support/checkstyle.xml b/dev-support/checkstyle.xml
index ec36760..45325bf 100644
--- a/dev-support/checkstyle.xml
+++ b/dev-support/checkstyle.xml
@@ -21,7 +21,7 @@
 "http://www.puppycrawl.com/dtds/configuration_1_3.dtd;>
 
 
-
+
 
 
 



[46/50] [abbrv] kylin git commit: KYLIN-1919 support embedded json format

2016-10-07 Thread mahongbin
KYLIN-1919 support embedded json format

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

Branch: refs/heads/orderedbytes
Commit: aa51ce0c3382d330ba5418b49eb669c964315f96
Parents: 792d4ee
Author: shaofengshi 
Authored: Thu Sep 29 16:04:46 2016 +0800
Committer: shaofengshi 
Committed: Thu Oct 6 14:44:05 2016 +0800

--
 .../java/org/apache/kylin/job/DeployUtil.java   |  2 +-
 .../kylin/provision/BuildCubeWithEngine.java|  2 +-
 .../kylin/provision/BuildCubeWithStream.java| 16 ++--
 .../java/org/apache/kylin/rest/DebugTomcat.java |  3 +-
 .../kylin/source/kafka/AbstractTimeParser.java  |  4 +-
 .../kylin/source/kafka/DateTimeParser.java  | 40 ++---
 .../kylin/source/kafka/DefaultTimeParser.java   |  4 +-
 .../kylin/source/kafka/StreamingParser.java | 41 -
 .../source/kafka/StringStreamingParser.java |  3 +-
 .../source/kafka/TimedJsonStreamParser.java | 95 
 .../kafka/diagnose/KafkaInputAnalyzer.java  |  6 +-
 11 files changed, 133 insertions(+), 83 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/aa51ce0c/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
--
diff --git a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java 
b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
index 9e9df05..be9b2a9 100644
--- a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -156,7 +156,7 @@ public class DeployUtil {
 for (ColumnDesc columnDesc : tableDesc.getColumns()) {
 tableColumns.add(columnDesc.getRef());
 }
-TimedJsonStreamParser timedJsonStreamParser = new 
TimedJsonStreamParser(tableColumns, "formatTs=true");
+TimedJsonStreamParser timedJsonStreamParser = new 
TimedJsonStreamParser(tableColumns, null);
 StringBuilder sb = new StringBuilder();
 for (String json : data) {
 List rowColumns = 
timedJsonStreamParser.parse(ByteBuffer.wrap(json.getBytes())).getData();

http://git-wip-us.apache.org/repos/asf/kylin/blob/aa51ce0c/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
--
diff --git 
a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java 
b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
index 31cf0eb..971b293 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
@@ -84,7 +84,7 @@ public class BuildCubeWithEngine {
 afterClass();
 logger.info("Going to exit");
 System.exit(0);
-} catch (Exception e) {
+} catch (Throwable e) {
 logger.error("error", e);
 System.exit(1);
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/aa51ce0c/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
--
diff --git 
a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java 
b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
index 58715f1..f8805a6 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
@@ -234,6 +234,8 @@ public class BuildCubeWithStream {
 segments = cubeManager.getCube(cubeName).getSegments();
 Assert.assertTrue(segments.size() == 1);
 }
+
+logger.info("Build is done");
 }
 
 
@@ -309,20 +311,22 @@ public class BuildCubeWithStream {
 }
 
 public static void main(String[] args) throws Exception {
+BuildCubeWithStream buildCubeWithStream = null;
 try {
 beforeClass();
-
-BuildCubeWithStream buildCubeWithStream = new 
BuildCubeWithStream();
+buildCubeWithStream = new BuildCubeWithStream();
 buildCubeWithStream.before();
 buildCubeWithStream.build();
-logger.info("Build is done");
-buildCubeWithStream.after();
-afterClass();
 logger.info("Going to exit");
 System.exit(0);
-} catch (Exception e) {
+} catch (Throwable e) {
 logger.error("error", e);
 System.exit(1);
+} finally {
+if 

[12/50] [abbrv] kylin git commit: KYLIN-2019: Enable Apache Licence checker for Checkstyle

2016-10-07 Thread mahongbin
http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/source-kafka/src/main/java/org/apache/kylin/source/kafka/StringStreamingParser.java
--
diff --git 
a/source-kafka/src/main/java/org/apache/kylin/source/kafka/StringStreamingParser.java
 
b/source-kafka/src/main/java/org/apache/kylin/source/kafka/StringStreamingParser.java
index d67..cea8e0b 100644
--- 
a/source-kafka/src/main/java/org/apache/kylin/source/kafka/StringStreamingParser.java
+++ 
b/source-kafka/src/main/java/org/apache/kylin/source/kafka/StringStreamingParser.java
@@ -1,36 +1,20 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/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 20c57a9..d3530f1 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
@@ -1,36 +1,20 @@
 /*
- *
- *
- *  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.
- *
- * /
- */
+ * 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;
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/source-kafka/src/main/java/org/apache/kylin/source/kafka/TopicMeta.java
--
diff --git 

[27/50] [abbrv] kylin git commit: Revert "Revert "KYLIN-1726 Scalable streaming cubing""

2016-10-07 Thread mahongbin
Revert "Revert "KYLIN-1726 Scalable streaming cubing""

This reverts commit 506cd783132023a06f1669ad248b74bf9d96d0e1.


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

Branch: refs/heads/orderedbytes
Commit: 8431af45528abb2d39a69b5e762712983573e5a6
Parents: a00d1e3
Author: shaofengshi 
Authored: Sat Sep 24 14:55:59 2016 +0800
Committer: shaofengshi 
Committed: Tue Sep 27 10:17:40 2016 +0800

--
 .../java/org/apache/kylin/job/DeployUtil.java   |  35 ++-
 .../kylin/job/streaming/Kafka10DataLoader.java  |  80 +++
 .../apache/kylin/common/KylinConfigBase.java|   1 +
 .../java/org/apache/kylin/cube/CubeSegment.java |   1 +
 .../java/org/apache/kylin/cube/ISegment.java|  39 
 .../cube/gridtable/SegmentGTStartAndEnd.java|   2 +-
 .../cube/model/CubeJoinedFlatTableDesc.java |   6 +
 .../cube/model/CubeJoinedFlatTableEnrich.java   |   6 +
 .../apache/kylin/gridtable/ScannerWorker.java   |   2 +-
 .../metadata/model/IJoinedFlatTableDesc.java|   2 +
 .../apache/kylin/metadata/model/ISegment.java   |  36 +++
 .../kylin/engine/mr/BatchMergeJobBuilder2.java  |   3 +
 .../org/apache/kylin/engine/mr/IMRInput.java|  10 +
 .../java/org/apache/kylin/engine/mr/MRUtil.java |   4 +
 .../test_streaming_table_model_desc.json|   6 +-
 .../kylin/provision/BuildCubeWithStream.java| 218 +-
 .../org/apache/kylin/provision/MockKafka.java   | 191 
 .../apache/kylin/provision/NetworkUtils.java|  52 +
 pom.xml |   2 +-
 .../apache/kylin/source/hive/HiveMRInput.java   |  11 +
 source-kafka/pom.xml|  13 +-
 .../apache/kylin/source/kafka/KafkaMRInput.java | 221 +++
 .../apache/kylin/source/kafka/KafkaSource.java  |  57 +
 .../kylin/source/kafka/KafkaStreamingInput.java |  17 +-
 .../kylin/source/kafka/MergeOffsetStep.java |  89 
 .../kylin/source/kafka/SeekOffsetStep.java  | 119 ++
 .../kylin/source/kafka/UpdateTimeRangeStep.java | 108 +
 .../source/kafka/config/KafkaClusterConfig.java |   3 +-
 .../source/kafka/hadoop/KafkaFlatTableJob.java  | 165 ++
 .../kafka/hadoop/KafkaFlatTableMapper.java  |  51 +
 .../source/kafka/hadoop/KafkaInputFormat.java   |  98 
 .../kafka/hadoop/KafkaInputRecordReader.java| 166 ++
 .../source/kafka/hadoop/KafkaInputSplit.java| 102 +
 .../kylin/source/kafka/util/KafkaClient.java| 115 ++
 .../source/kafka/util/KafkaOffsetMapping.java   |  97 
 .../kylin/source/kafka/util/KafkaRequester.java |   7 +-
 .../kylin/source/kafka/util/KafkaUtils.java |   3 +-
 .../hbase/cube/v2/CubeHBaseEndpointRPC.java |   2 +-
 .../storage/hbase/cube/v2/CubeHBaseRPC.java |   2 +-
 .../storage/hbase/cube/v2/CubeHBaseScanRPC.java |   2 +-
 40 files changed, 2024 insertions(+), 120 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/8431af45/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
--
diff --git a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java 
b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
index 8c64f91..9b282e3 100644
--- a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -143,14 +143,12 @@ public class DeployUtil {
 deployHiveTables();
 }
 
-public static void prepareTestDataForStreamingCube(long startTime, long 
endTime, String cubeName, StreamDataLoader streamDataLoader) throws IOException 
{
+public static void prepareTestDataForStreamingCube(long startTime, long 
endTime, int numberOfRecords, String cubeName, StreamDataLoader 
streamDataLoader) throws IOException {
 CubeInstance cubeInstance = 
CubeManager.getInstance(KylinConfig.getInstanceFromEnv()).getCube(cubeName);
-List data = StreamingTableDataGenerator.generate(1, 
startTime, endTime, cubeInstance.getFactTable());
-List data2 = StreamingTableDataGenerator.generate(10, endTime, 
endTime + 30, cubeInstance.getFactTable());
+List data = 
StreamingTableDataGenerator.generate(numberOfRecords, startTime, endTime, 
cubeInstance.getFactTable());
 TableDesc tableDesc = cubeInstance.getFactTableDesc();
 //load into kafka
 streamDataLoader.loadIntoKafka(data);
-streamDataLoader.loadIntoKafka(data2);
 logger.info("Write {} messages into {}", data.size(), 
streamDataLoader.toString());
 
 //csv 

[48/50] [abbrv] kylin git commit: minor, a cautious check in TrieDictionaryBuilder

2016-10-07 Thread mahongbin
minor, a cautious check in TrieDictionaryBuilder


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

Branch: refs/heads/orderedbytes
Commit: 483e63654dbff10a789644f9b8e8039466ae393c
Parents: cd9de01
Author: Yang Li 
Authored: Fri Oct 7 12:39:09 2016 +0800
Committer: Yang Li 
Committed: Fri Oct 7 12:40:12 2016 +0800

--
 .../src/main/java/org/apache/kylin/dict/TrieDictionaryBuilder.java | 2 ++
 1 file changed, 2 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/483e6365/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryBuilder.java
--
diff --git 
a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryBuilder.java
 
b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryBuilder.java
index 1271483..c4895c3 100644
--- 
a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryBuilder.java
+++ 
b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryBuilder.java
@@ -417,6 +417,8 @@ public class TrieDictionaryBuilder {
 int sizeNoValuesBeneath = stats.mbpn_sizeNoValueBeneath;
 int sizeChildOffset = stats.mbpn_sizeChildOffset;
 
+if (stats.mbpn_footprint <= 0) // must never happen, but let us be 
cautious
+throw new IllegalStateException("Too big dictionary, dictionary 
cannot be bigger than 2GB");
 if (stats.mbpn_footprint > _2GB)
 throw new RuntimeException("Too big dictionary, dictionary cannot 
be bigger than 2GB");
 



[07/50] [abbrv] kylin git commit: For integration test, don’t use snappy compress for outputformat

2016-10-07 Thread mahongbin
For integration test, don’t use snappy compress for outputformat

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

Branch: refs/heads/orderedbytes
Commit: 0861bfb0312d344267eaf5772ed1b3011f778301
Parents: b95e967
Author: shaofengshi 
Authored: Mon Sep 26 11:55:31 2016 +0800
Committer: shaofengshi 
Committed: Mon Sep 26 11:55:31 2016 +0800

--
 examples/test_case_data/sandbox/kylin_hive_conf.xml | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/0861bfb0/examples/test_case_data/sandbox/kylin_hive_conf.xml
--
diff --git a/examples/test_case_data/sandbox/kylin_hive_conf.xml 
b/examples/test_case_data/sandbox/kylin_hive_conf.xml
index 4ee5f6b..593d4f8 100644
--- a/examples/test_case_data/sandbox/kylin_hive_conf.xml
+++ b/examples/test_case_data/sandbox/kylin_hive_conf.xml
@@ -45,12 +45,13 @@
 org.apache.hadoop.io.compress.SnappyCodec
 
 
+
 
 hive.merge.size.per.task
 3200



[22/50] [abbrv] kylin git commit: Revert "Revert "change to upper case""

2016-10-07 Thread mahongbin
Revert "Revert "change to upper case""

This reverts commit d1e979b4995597ae2ac3fbeb88ba1902d7296782.


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

Branch: refs/heads/orderedbytes
Commit: de2f4e2a395af57a42aba43f7233bd1bbdd0f6d0
Parents: 8136380
Author: shaofengshi 
Authored: Sat Sep 24 14:57:22 2016 +0800
Committer: shaofengshi 
Committed: Tue Sep 27 10:17:40 2016 +0800

--
 .../kafka/DEFAULT.STREAMING_TABLE.json  | 21 
 .../streaming/DEFAULT.STREAMING_TABLE.json  |  6 ++
 2 files changed, 27 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/de2f4e2a/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json
--
diff --git 
a/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json 
b/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json
new file mode 100644
index 000..6a64cce
--- /dev/null
+++ b/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json
@@ -0,0 +1,21 @@
+{
+ 
+  "uuid": "8b2b9dfe-777c-4d39-bf89-8472ec909193",
+  "name": "DEFAULT.STREAMING_TABLE",
+  "topic": "test_streaming_table_topic_xyz",
+  "timeout": 6,
+  "bufferSize": 65536,
+  "parserName": "org.apache.kylin.source.kafka.TimedJsonStreamParser",
+  "last_modified": 0,
+  "clusters": [
+{
+  "brokers": [
+{
+  "id": 0,
+  "host": "sandbox",
+  "port": 6667
+}
+  ]
+}
+  ]
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/de2f4e2a/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json
--
diff --git 
a/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json 
b/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json
new file mode 100644
index 000..85a477b
--- /dev/null
+++ b/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json
@@ -0,0 +1,6 @@
+{
+  "uuid": "8b2b9dfe-777c-4d39-bf89-8472ec909193",
+  "name": "DEFAULT.STREAMING_TABLE",
+  "type": "kafka",
+  "last_modified": 0
+}



[49/50] [abbrv] kylin git commit: KYLIN-2051 Fix potentially ineffective call in ZipFileUtils class

2016-10-07 Thread mahongbin
KYLIN-2051 Fix potentially ineffective call in ZipFileUtils class


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

Branch: refs/heads/orderedbytes
Commit: 4fae26781af999c7274c176d4bf6e87e28bb9bfe
Parents: 483e636
Author: lidongsjtu 
Authored: Fri Oct 7 18:22:39 2016 +0800
Committer: lidongsjtu 
Committed: Fri Oct 7 18:22:39 2016 +0800

--
 .../apache/kylin/common/util/ZipFileUtils.java  | 57 
 1 file changed, 34 insertions(+), 23 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/4fae2678/core-common/src/main/java/org/apache/kylin/common/util/ZipFileUtils.java
--
diff --git 
a/core-common/src/main/java/org/apache/kylin/common/util/ZipFileUtils.java 
b/core-common/src/main/java/org/apache/kylin/common/util/ZipFileUtils.java
index 401c556..7ee770d 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/ZipFileUtils.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/ZipFileUtils.java
@@ -39,32 +39,41 @@ public class ZipFileUtils {
 if (!validateZipFilename(zipFilename)) {
 throw new RuntimeException("Zipfile must end with .zip");
 }
-ZipOutputStream zipFile = new ZipOutputStream(new 
FileOutputStream(zipFilename));
-compressDirectoryToZipfile(normDir(new File(sourceDir).getParent()), 
normDir(sourceDir), zipFile);
-IOUtils.closeQuietly(zipFile);
+ZipOutputStream zipFile = null;
+try {
+zipFile = new ZipOutputStream(new FileOutputStream(zipFilename));
+compressDirectoryToZipfile(normDir(new 
File(sourceDir).getParent()), normDir(sourceDir), zipFile);
+} finally {
+IOUtils.closeQuietly(zipFile);
+}
 }
 
 public static void decompressZipfileToDirectory(String zipFileName, File 
outputFolder) throws IOException {
+ZipInputStream zipInputStream = null;
+try {
+zipInputStream = new ZipInputStream(new 
FileInputStream(zipFileName));
+ZipEntry zipEntry = null;
+while ((zipEntry = zipInputStream.getNextEntry()) != null) {
+logger.info("decompressing " + zipEntry.getName() + " is 
directory:" + zipEntry.isDirectory() + " available: " + 
zipInputStream.available());
 
-ZipInputStream zipInputStream = new ZipInputStream(new 
FileInputStream(zipFileName));
-ZipEntry zipEntry = null;
-while ((zipEntry = zipInputStream.getNextEntry()) != null) {
-logger.info("decompressing " + zipEntry.getName() + " is 
directory:" + zipEntry.isDirectory() + " available: " + 
zipInputStream.available());
-
-File temp = new File(outputFolder, zipEntry.getName());
-if (zipEntry.isDirectory()) {
-temp.mkdirs();
-} else {
-temp.getParentFile().mkdirs();
-temp.createNewFile();
-temp.setLastModified(zipEntry.getTime());
-FileOutputStream outputStream = new FileOutputStream(temp);
-IOUtils.copy(zipInputStream, outputStream);
-IOUtils.closeQuietly(outputStream);
+File temp = new File(outputFolder, zipEntry.getName());
+if (zipEntry.isDirectory()) {
+temp.mkdirs();
+} else {
+temp.getParentFile().mkdirs();
+temp.createNewFile();
+temp.setLastModified(zipEntry.getTime());
+FileOutputStream outputStream = new FileOutputStream(temp);
+try {
+IOUtils.copy(zipInputStream, outputStream);
+} finally {
+IOUtils.closeQuietly(outputStream);
+}
+}
 }
+} finally {
+IOUtils.closeQuietly(zipInputStream);
 }
-IOUtils.closeQuietly(zipInputStream);
-
 }
 
 private static void compressDirectoryToZipfile(String rootDir, String 
sourceDir, ZipOutputStream out) throws IOException {
@@ -75,10 +84,12 @@ public class ZipFileUtils {
 ZipEntry entry = new 
ZipEntry(normDir(StringUtils.isEmpty(rootDir) ? sourceDir : 
sourceDir.replace(rootDir, "")) + sourceFile.getName());
 entry.setTime(sourceFile.lastModified());
 out.putNextEntry(entry);
-
 FileInputStream in = new FileInputStream(sourceDir + 
sourceFile.getName());
-IOUtils.copy(in, out);
-IOUtils.closeQuietly(in);
+ 

[38/50] [abbrv] kylin git commit: enlarge the job interval in BuildCubeWithStream

2016-10-07 Thread mahongbin
enlarge the job interval in BuildCubeWithStream


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

Branch: refs/heads/orderedbytes
Commit: dfb4176531046e212649dce89aebb8d091fc5689
Parents: 1e04981
Author: shaofengshi 
Authored: Fri Sep 30 11:02:09 2016 +0800
Committer: shaofengshi 
Committed: Fri Sep 30 11:02:09 2016 +0800

--
 .../java/org/apache/kylin/provision/BuildCubeWithStream.java | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/dfb41765/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
--
diff --git 
a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java 
b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
index 23d7ca8..b48f75b 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
@@ -22,9 +22,7 @@ import java.io.File;
 import java.io.IOException;
 import java.text.ParseException;
 import java.text.SimpleDateFormat;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.Random;
 import java.util.TimeZone;
 import java.util.UUID;
@@ -179,7 +177,7 @@ public class BuildCubeWithStream {
 
 List futures = Lists.newArrayList();
 for (int i = 0; i < 5; i++) {
-Thread.sleep(2 * 60 * 1000); // wait for new messages
+Thread.sleep(5 * 60 * 1000); // wait for new messages
 FutureTask futureTask = new FutureTask(new 
Callable() {
 @Override
 public ExecutableState call() {



[21/50] [abbrv] kylin git commit: Revert "Revert "KYLIN-1726 allow job discard itself""

2016-10-07 Thread mahongbin
Revert "Revert "KYLIN-1726 allow job discard itself""

This reverts commit 3ae2549ba89e3a2c8ed94a2089678227cf78312d.


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

Branch: refs/heads/orderedbytes
Commit: f0de02392f4808e08432417cb00f2ab5fa829055
Parents: de2f4e2
Author: shaofengshi 
Authored: Sat Sep 24 14:57:36 2016 +0800
Committer: shaofengshi 
Committed: Tue Sep 27 10:17:40 2016 +0800

--
 .../java/org/apache/kylin/cube/CubeManager.java | 14 ++-
 .../kylin/job/execution/AbstractExecutable.java |  2 +
 .../job/execution/DefaultChainedExecutable.java |  2 +
 .../kylin/job/execution/ExecuteResult.java  |  4 ++
 .../kylin/job/DiscardedTestExecutable.java  | 41 
 .../impl/threadpool/DefaultSchedulerTest.java   | 16 
 6 files changed, 68 insertions(+), 11 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/f0de0239/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 5a4b07c..3a327f9 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
@@ -357,34 +357,26 @@ public class CubeManager implements IRealizationProvider {
 Iterator iterator = newSegs.iterator();
 while (iterator.hasNext()) {
 CubeSegment currentSeg = iterator.next();
-boolean found = false;
 for (CubeSegment toRemoveSeg : update.getToRemoveSegs()) {
 if (currentSeg.getUuid().equals(toRemoveSeg.getUuid())) {
+logger.info("Remove segment " + currentSeg.toString());
+
toRemoveResources.add(currentSeg.getStatisticsResourcePath());
 iterator.remove();
-
toRemoveResources.add(toRemoveSeg.getStatisticsResourcePath());
-found = true;
+break;
 }
 }
-if (found == false) {
-logger.error("Segment '" + currentSeg.getName() + "' 
doesn't exist for remove.");
-}
 }
 
 }
 
 if (update.getToUpdateSegs() != null) {
 for (CubeSegment segment : update.getToUpdateSegs()) {
-boolean found = false;
 for (int i = 0; i < newSegs.size(); i++) {
 if (newSegs.get(i).getUuid().equals(segment.getUuid())) {
 newSegs.set(i, segment);
-found = true;
 break;
 }
 }
-if (found == false) {
-logger.error("Segment '" + segment.getName() + "' doesn't 
exist for update.");
-}
 }
 }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/f0de0239/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
--
diff --git 
a/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java 
b/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
index 90e4d3c..b4ca469 100644
--- 
a/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
+++ 
b/core-job/src/main/java/org/apache/kylin/job/execution/AbstractExecutable.java
@@ -74,6 +74,8 @@ public abstract class AbstractExecutable implements 
Executable, Idempotent {
 if (!isDiscarded()) {
 if (result.succeed()) {
 executableManager.updateJobOutput(getId(), 
ExecutableState.SUCCEED, null, result.output());
+} else if (result.discarded()) {
+executableManager.updateJobOutput(getId(), 
ExecutableState.DISCARDED, null, result.output());
 } else {
 executableManager.updateJobOutput(getId(), 
ExecutableState.ERROR, null, result.output());
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/f0de0239/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
--
diff --git 
a/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
 
b/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
index 39a5f4f..5a57b05 100644

[13/50] [abbrv] kylin git commit: KYLIN-2019: Enable Apache Licence checker for Checkstyle

2016-10-07 Thread mahongbin
KYLIN-2019: Enable Apache Licence checker for Checkstyle

Signed-off-by: Yang Li 


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

Branch: refs/heads/orderedbytes
Commit: 8596af11e9f46b7be82009f446b8adcca0d4e499
Parents: abc6a74
Author: Yiming Liu 
Authored: Mon Sep 26 20:17:42 2016 +0800
Committer: Yang Li 
Committed: Mon Sep 26 21:45:37 2016 +0800

--
 .../main/config/assemblies/source-assembly.xml  | 117 ++
 checkstyle-suppressions.xml |  28 --
 checkstyle.xml  |  62 ---
 .../org/apache/kylin/common/KylinVersion.java   |  24 +-
 .../org/apache/kylin/common/util/Bytes.java |   5 +-
 .../common/util/MemoryBudgetController.java |  27 +-
 .../java/org/apache/kylin/common/util/Pair.java |   5 +-
 .../kylin/common/util/StreamingBatch.java   |  49 +--
 .../cube/gridtable/TrimmedCubeCodeSystem.java   |  32 +-
 .../inmemcubing/AbstractInMemCubeBuilder.java   |  28 +-
 .../cube/inmemcubing/CompoundCuboidWriter.java  |  27 +-
 .../cube/inmemcubing/ConcurrentDiskStore.java   |  27 +-
 .../cube/inmemcubing/DoggedCubeBuilder.java |  28 +-
 .../kylin/cube/inmemcubing/ICuboidWriter.java   |  27 +-
 .../cube/inmemcubing/InMemCubeBuilder.java  |  28 +-
 .../kylin/cube/inmemcubing/MemDiskStore.java|  27 +-
 .../org/apache/kylin/cube/util/CubingUtils.java |  49 +--
 .../org/apache/kylin/gridtable/IGTStorage.java  |  32 +-
 .../apache/kylin/gridtable/ScannerWorker.java   |  32 +-
 .../apache/kylin/gridtable/UnitTestSupport.java |  27 +-
 dev-support/checkstyle-java-header.txt  |  17 +
 dev-support/checkstyle-suppressions.xml |  28 ++
 dev-support/checkstyle.xml  |  67 
 .../kylin/engine/streaming/IStreamingInput.java |  49 +--
 .../engine/streaming/IStreamingOutput.java  |  49 +--
 .../streaming/OneOffStreamingBuilder.java   |  49 +--
 .../engine/streaming/StreamingBatchBuilder.java |  49 +--
 .../kylin/engine/streaming/StreamingConfig.java |  48 +--
 .../engine/streaming/StreamingManager.java  |  48 +--
 .../engine/streaming/cli/StreamingCLI.java  |  48 +--
 .../streaming/cube/StreamingCubeBuilder.java|  49 +--
 .../engine/streaming/util/StreamingUtils.java   |  49 +--
 pom.xml |   6 +-
 .../apache/kylin/source/hive/SchemaChecker.java |  18 +
 .../kylin/source/kafka/KafkaConfigManager.java  |  48 +--
 .../kylin/source/kafka/KafkaStreamingInput.java |  49 +--
 .../kylin/source/kafka/StreamingParser.java |  48 +--
 .../source/kafka/StringStreamingParser.java |  48 +--
 .../source/kafka/TimedJsonStreamParser.java |  48 +--
 .../apache/kylin/source/kafka/TopicMeta.java|  48 +--
 .../kylin/source/kafka/config/BrokerConfig.java |  48 +--
 .../kylin/source/kafka/config/KafkaConfig.java  |  48 +--
 .../kylin/source/kafka/util/KafkaRequester.java |  48 +--
 src/.settings/org.eclipse.core.resources.prefs  |   6 -
 src/.settings/org.eclipse.jdt.core.prefs| 386 ---
 src/.settings/org.eclipse.jdt.ui.prefs  |   7 -
 src/main/config/assemblies/source-assembly.xml  | 117 --
 .../checkstyle/checkstyle-java-header.txt   |  17 -
 src/main/config/checkstyle/checkstyle.xml   |  44 ---
 src/main/config/checkstyle/suppressions.xml |  32 --
 .../storage/hbase/steps/HBaseCuboidWriter.java  |  49 +--
 51 files changed, 788 insertions(+), 1558 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/8596af11/assembly/src/main/config/assemblies/source-assembly.xml
--
diff --git a/assembly/src/main/config/assemblies/source-assembly.xml 
b/assembly/src/main/config/assemblies/source-assembly.xml
new file mode 100644
index 000..23a215b
--- /dev/null
+++ b/assembly/src/main/config/assemblies/source-assembly.xml
@@ -0,0 +1,117 @@
+
+
+
+src
+
+zip
+tar.gz
+
+
+
+
+
+.
+.
+true
+
+
+
%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/).*${project.build.directory}.*]
+
+
+
+
+
+
%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?maven-eclipse\.xml]
+
+
%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.project]
+
+
%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?\.classpath]
+
+

[35/50] [abbrv] kylin git commit: minor add a blank line in create_sample_table.sql for beeline

2016-10-07 Thread mahongbin
minor add a blank line in create_sample_table.sql for beeline


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

Branch: refs/heads/orderedbytes
Commit: 0fd4ad587e3710324eaf58266b64f202eeabfa7e
Parents: e500726
Author: shaofengshi 
Authored: Wed Sep 28 17:51:36 2016 +0800
Committer: shaofengshi 
Committed: Wed Sep 28 17:51:36 2016 +0800

--
 examples/sample_cube/create_sample_tables.sql | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/0fd4ad58/examples/sample_cube/create_sample_tables.sql
--
diff --git a/examples/sample_cube/create_sample_tables.sql 
b/examples/sample_cube/create_sample_tables.sql
index d83566c..5ed7b9b 100644
--- a/examples/sample_cube/create_sample_tables.sql
+++ b/examples/sample_cube/create_sample_tables.sql
@@ -187,4 +187,4 @@ STORED AS TEXTFILE;
 
 LOAD DATA INPATH '/tmp/kylin/sample_cube/data/DEFAULT.KYLIN_SALES.csv' 
OVERWRITE INTO TABLE DEFAULT.KYLIN_SALES;
 LOAD DATA INPATH '/tmp/kylin/sample_cube/data/DEFAULT.KYLIN_CAL_DT.csv' 
OVERWRITE INTO TABLE DEFAULT.KYLIN_CAL_DT;
-LOAD DATA INPATH 
'/tmp/kylin/sample_cube/data/DEFAULT.KYLIN_CATEGORY_GROUPINGS.csv' OVERWRITE 
INTO TABLE DEFAULT.KYLIN_CATEGORY_GROUPINGS;
\ No newline at end of file
+LOAD DATA INPATH 
'/tmp/kylin/sample_cube/data/DEFAULT.KYLIN_CATEGORY_GROUPINGS.csv' OVERWRITE 
INTO TABLE DEFAULT.KYLIN_CATEGORY_GROUPINGS;



[15/50] [abbrv] kylin git commit: minor, add dev-support/sync_hbase_cdh_branches.cmd

2016-10-07 Thread mahongbin
minor, add dev-support/sync_hbase_cdh_branches.cmd


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

Branch: refs/heads/orderedbytes
Commit: afd3e61eedea2ce175e3fb2844b6118aabe25a6f
Parents: 05afca7
Author: Yang Li 
Authored: Tue Sep 27 08:38:42 2016 +0800
Committer: Yang Li 
Committed: Tue Sep 27 08:38:42 2016 +0800

--
 dev-support/sync_hbase_cdh_branches.cmd | 19 +++
 1 file changed, 19 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/afd3e61e/dev-support/sync_hbase_cdh_branches.cmd
--
diff --git a/dev-support/sync_hbase_cdh_branches.cmd 
b/dev-support/sync_hbase_cdh_branches.cmd
new file mode 100644
index 000..0febfbb
--- /dev/null
+++ b/dev-support/sync_hbase_cdh_branches.cmd
@@ -0,0 +1,19 @@
+git checkout master
+git pull
+git reset apache/master --hard
+
+git checkout apache/1.5.x-HBase1.x
+git format-patch -1
+git checkout master
+git am -3 --ignore-whitespace 
0001-KYLIN-1528-Create-a-branch-for-v1.5-with-HBase-1.x-A.patch
+git push apache master:1.5.x-HBase1.x -f
+rm 0001-KYLIN-1528-Create-a-branch-for-v1.5-with-HBase-1.x-A.patch
+
+git checkout apache/1.5.x-CDH5.7
+git format-patch -1
+git checkout master
+git am -3 --ignore-whitespace 0001-KYLIN-1672-support-kylin-on-cdh-5.7.patch
+git push apache master:1.5.x-CDH5.7 -f
+rm 0001-KYLIN-1672-support-kylin-on-cdh-5.7.patch
+
+git reset apache/master --hard



[39/50] [abbrv] kylin git commit: fix it, run StorageCleanup after finish BuildCubeWithStream

2016-10-07 Thread mahongbin
fix it, run StorageCleanup after finish BuildCubeWithStream

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

Branch: refs/heads/orderedbytes
Commit: 3a6b1f2d1cd1a42837353e52e3fa219c0d5a314b
Parents: dfb4176
Author: shaofengshi 
Authored: Fri Sep 30 14:04:12 2016 +0800
Committer: shaofengshi 
Committed: Fri Sep 30 14:04:12 2016 +0800

--
 .../org/apache/kylin/provision/BuildCubeWithStream.java  | 11 ++-
 1 file changed, 10 insertions(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/3a6b1f2d/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
--
diff --git 
a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java 
b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
index b48f75b..58715f1 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
@@ -35,6 +35,7 @@ import java.util.concurrent.TimeUnit;
 import com.google.common.collect.Lists;
 import org.I0Itec.zkclient.ZkConnection;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.util.ToolRunner;
 import org.apache.kafka.common.requests.MetadataResponse;
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.ClassUtil;
@@ -58,6 +59,7 @@ import org.apache.kylin.metadata.model.SegmentStatusEnum;
 import org.apache.kylin.source.kafka.KafkaConfigManager;
 import org.apache.kylin.source.kafka.config.BrokerConfig;
 import org.apache.kylin.source.kafka.config.KafkaConfig;
+import org.apache.kylin.storage.hbase.util.StorageCleanupJob;
 import org.apache.kylin.storage.hbase.util.ZookeeperJobLock;
 import org.junit.Assert;
 import org.slf4j.Logger;
@@ -177,7 +179,7 @@ public class BuildCubeWithStream {
 
 List futures = Lists.newArrayList();
 for (int i = 0; i < 5; i++) {
-Thread.sleep(5 * 60 * 1000); // wait for new messages
+Thread.sleep(2 * 60 * 1000); // wait for new messages
 FutureTask futureTask = new FutureTask(new 
Callable() {
 @Override
 public ExecutableState call() {
@@ -276,6 +278,7 @@ public class BuildCubeWithStream {
 }
 
 public static void afterClass() throws Exception {
+cleanupOldStorage();
 HBaseMetadataTestCase.staticCleanupTestMetadata();
 }
 
@@ -299,6 +302,12 @@ public class BuildCubeWithStream {
 }
 }
 
+private static void cleanupOldStorage() throws Exception {
+String[] args = { "--delete", "true" };
+StorageCleanupJob cli = new StorageCleanupJob();
+cli.execute(args);
+}
+
 public static void main(String[] args) throws Exception {
 try {
 beforeClass();



[43/50] [abbrv] kylin git commit: minor, enhance dev-support scripts

2016-10-07 Thread mahongbin
minor, enhance dev-support scripts


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

Branch: refs/heads/orderedbytes
Commit: 792d4ee3fd3f12b63a34a47bb90ed7b87449f15a
Parents: 937cc54
Author: Yang Li 
Authored: Tue Oct 4 16:55:46 2016 +0800
Committer: Yang Li 
Committed: Tue Oct 4 16:55:46 2016 +0800

--
 dev-support/sync_hbase_cdh_branches.cmd | 43 --
 dev-support/sync_hbase_cdh_branches.sh  | 45 
 2 files changed, 45 insertions(+), 43 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/792d4ee3/dev-support/sync_hbase_cdh_branches.cmd
--
diff --git a/dev-support/sync_hbase_cdh_branches.cmd 
b/dev-support/sync_hbase_cdh_branches.cmd
deleted file mode 100644
index ed828ca..000
--- a/dev-support/sync_hbase_cdh_branches.cmd
+++ /dev/null
@@ -1,43 +0,0 @@
-#!/bin/bash
-
-# 
-
-base=master
-
-# 
-
-set -o pipefail  # trace ERR through pipes
-set -o errtrace  # trace ERR through 'time command' and other functions
-function error() {
-   SCRIPT="$0"   # script name
-   LASTLINE="$1" # line of error occurrence
-   LASTERR="$2"  # error code
-   echo "ERROR exit from ${SCRIPT} : line ${LASTLINE} with exit code 
${LASTERR}"
-   exit 1
-}
-trap 'error ${LINENO} ${?}' ERR
-
-# 
-
-git fetch apache
-git checkout apache/$base-hbase1.x
-git format-patch -1
-git checkout apache/$base-cdh1.x
-git format-patch -1
-
-git checkout apache/$base
-git checkout -b tmp
-git reset apache/$base --hard
-
-git am -3 --ignore-whitespace 
0001-KYLIN-1528-Create-a-branch-for-v1.5-with-HBase-1.x-A.patch
-#git push apache tmp:$base-hbase1.x -f
-rm 0001-KYLIN-1528-Create-a-branch-for-v1.5-with-HBase-1.x-A.patch
-
-git am -3 --ignore-whitespace 0001-KYLIN-1672-support-kylin-on-cdh-5.7.patch
-#git push apache tmp:$base-cdh5.7 -f
-rm 0001-KYLIN-1672-support-kylin-on-cdh-5.7.patch
-
-# clean up
-git checkout master
-git reset apache/master --hard
-git checkout -b tmp

http://git-wip-us.apache.org/repos/asf/kylin/blob/792d4ee3/dev-support/sync_hbase_cdh_branches.sh
--
diff --git a/dev-support/sync_hbase_cdh_branches.sh 
b/dev-support/sync_hbase_cdh_branches.sh
new file mode 100644
index 000..f52284a
--- /dev/null
+++ b/dev-support/sync_hbase_cdh_branches.sh
@@ -0,0 +1,45 @@
+#!/bin/bash
+
+# 
+
+base=master
+
+# 
+
+set -o pipefail  # trace ERR through pipes
+set -o errtrace  # trace ERR through 'time command' and other functions
+function error() {
+   SCRIPT="$0"   # script name
+   LASTLINE="$1" # line of error occurrence
+   LASTERR="$2"  # error code
+   echo "ERROR exit from ${SCRIPT} : line ${LASTLINE} with exit code 
${LASTERR}"
+   exit 1
+}
+trap 'error ${LINENO} ${?}' ERR
+
+# 
+
+git fetch apache
+git checkout apache/$base-hbase1.x
+git format-patch -1
+git checkout apache/$base-cdh5.7
+git format-patch -1
+
+git checkout apache/$base
+git checkout -b tmp
+git reset apache/$base --hard
+
+git am -3 --ignore-whitespace 
0001-KYLIN-1528-Create-a-branch-for-v1.5-with-HBase-1.x-A.patch
+mvn clean compile -DskipTests
+git push apache tmp:$base-hbase1.x -f
+rm 0001-KYLIN-1528-Create-a-branch-for-v1.5-with-HBase-1.x-A.patch
+
+git am -3 --ignore-whitespace 0001-KYLIN-1672-support-kylin-on-cdh-5.7.patch
+mvn clean compile -DskipTests
+git push apache tmp:$base-cdh5.7 -f
+rm 0001-KYLIN-1672-support-kylin-on-cdh-5.7.patch
+
+# clean up
+git checkout master
+git reset apache/master --hard
+git branch -D tmp



[19/50] [abbrv] kylin git commit: Revert "Revert "KYLIN-1818 change kafka dependency to provided""

2016-10-07 Thread mahongbin
Revert "Revert "KYLIN-1818 change kafka dependency to provided""

This reverts commit dee8f2d073d225a3b8189c66170fd310c1a8d221.


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

Branch: refs/heads/orderedbytes
Commit: c738f0f265737dd5849f9342fd694e3cef3344b8
Parents: f89e35f
Author: shaofengshi 
Authored: Sat Sep 24 14:56:32 2016 +0800
Committer: shaofengshi 
Committed: Tue Sep 27 10:17:40 2016 +0800

--
 build/bin/find-kafka-dependency.sh  | 12 ++--
 build/bin/kylin.sh  |  2 ++
 .../kylin/engine/mr/common/AbstractHadoopJob.java   |  4 ++--
 3 files changed, 10 insertions(+), 8 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/c738f0f2/build/bin/find-kafka-dependency.sh
--
diff --git a/build/bin/find-kafka-dependency.sh 
b/build/bin/find-kafka-dependency.sh
index c6b9c24..7349360 100644
--- a/build/bin/find-kafka-dependency.sh
+++ b/build/bin/find-kafka-dependency.sh
@@ -32,20 +32,20 @@ then
 fi
 
 # works for kafka 9+
-kafka_client=`find -L "$(dirname $kafka_home)" -name 
'kafka-clients-[a-z0-9A-Z\.-]*.jar' ! -name '*doc*' ! -name '*test*' ! -name 
'*sources*' ''-printf '%p:' | sed 's/:$//'`
-if [ -z "$kafka_client" ]
+kafka_dependency=`find -L $kafka_home -name 
'kafka-clients-[a-z0-9A-Z\.-]*.jar' ! -name '*doc*' ! -name '*test*' ! -name 
'*sources*' ''-printf '%p:' | sed 's/:$//'`
+if [ -z "$kafka_dependency" ]
 then
 # works for kafka 8
-kafka_broker=`find -L "$(dirname $kafka_home)" -name 
'kafka_[a-z0-9A-Z\.-]*.jar' ! -name '*doc*' ! -name '*test*' ! -name 
'*sources*' ''-printf '%p:' | sed 's/:$//'`
-if [ -z "$kafka_broker" ]
+kafka_dependency=`find -L $kafka_home -name 'kafka_[a-z0-9A-Z\.-]*.jar' ! 
-name '*doc*' ! -name '*test*' ! -name '*sources*' ''-printf '%p:' | sed 
's/:$//'`
+if [ -z "$kafka_dependency" ]
 then
 echo "kafka client lib not found"
 exit 1
 else
-echo "kafka dependency: $kafka_broker"
+echo "kafka dependency: $kafka_dependency"
 export kafka_dependency
 fi
 else
-echo "kafka dependency: $kafka_client"
+echo "kafka dependency: $kafka_dependency"
 export kafka_dependency
 fi

http://git-wip-us.apache.org/repos/asf/kylin/blob/c738f0f2/build/bin/kylin.sh
--
diff --git a/build/bin/kylin.sh b/build/bin/kylin.sh
index 201b5b6..7a9d2a1 100644
--- a/build/bin/kylin.sh
+++ b/build/bin/kylin.sh
@@ -31,6 +31,7 @@ function retrieveDependency() {
 #retrive $hive_dependency and $hbase_dependency
 source ${dir}/find-hive-dependency.sh
 source ${dir}/find-hbase-dependency.sh
+source ${dir}/find-kafka-dependency.sh
 
 #retrive $KYLIN_EXTRA_START_OPTS
 if [ -f "${dir}/setenv.sh" ]
@@ -106,6 +107,7 @@ then
 -Djava.io.tmpdir=${tomcat_root}/temp  \
 -Dkylin.hive.dependency=${hive_dependency} \
 -Dkylin.hbase.dependency=${hbase_dependency} \
+-Dkylin.kafka.dependency=${kafka_dependency} \
 -Dkylin.rest.address=${kylin_rest_address} \
 -Dspring.profiles.active=${spring_profile} \
 org.apache.hadoop.util.RunJar ${tomcat_root}/bin/bootstrap.jar  
org.apache.catalina.startup.Bootstrap start >> ${KYLIN_HOME}/logs/kylin.out 
2>&1 & echo $! > ${KYLIN_HOME}/pid &

http://git-wip-us.apache.org/repos/asf/kylin/blob/c738f0f2/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
--
diff --git 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
index af2ed9f..a138eec 100644
--- 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
+++ 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/AbstractHadoopJob.java
@@ -226,11 +226,11 @@ public abstract class AbstractHadoopJob extends 
Configured implements Tool {
 }
 }
 
-// for hive dependencies
+// for kafka dependencies
 if (kylinKafkaDependency != null) {
 kylinKafkaDependency = kylinKafkaDependency.replace(":", ",");
 
-logger.info("Kafka Dependencies Before Filtered: " + 
kylinHiveDependency);
+logger.info("Kafka Dependencies Before Filtered: " + 
kylinKafkaDependency);
 
 if (kylinDependency.length() > 0)
 kylinDependency.append(",");



[25/50] [abbrv] kylin git commit: Revert "Revert "refactor BuildCubeWithStream""

2016-10-07 Thread mahongbin
Revert "Revert "refactor BuildCubeWithStream""

This reverts commit 8e9c4550bb562b497442b17eec6485ae96e848d8.


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

Branch: refs/heads/orderedbytes
Commit: be18158dcc5ce739c272b9345d3b2296c3936ee3
Parents: 8cbffb4
Author: shaofengshi 
Authored: Sat Sep 24 14:58:43 2016 +0800
Committer: shaofengshi 
Committed: Tue Sep 27 10:17:40 2016 +0800

--
 .../java/org/apache/kylin/job/DeployUtil.java   |   7 +-
 .../kylin/provision/BuildCubeWithStream.java|  10 +-
 .../kylin/provision/BuildCubeWithStream2.java   | 145 +--
 3 files changed, 12 insertions(+), 150 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/be18158d/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
--
diff --git a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java 
b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
index 9b282e3..9e9df05 100644
--- a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -187,6 +187,7 @@ public class DeployUtil {
 File tmpFile = File.createTempFile(factTableName, "csv");
 FileOutputStream out = new FileOutputStream(tmpFile);
 
+InputStream tempIn = null;
 try {
 if (store.exists(factTablePath)) {
 InputStream oldContent = 
store.getResource(factTablePath).inputStream;
@@ -194,13 +195,15 @@ public class DeployUtil {
 }
 IOUtils.copy(in, out);
 IOUtils.closeQuietly(in);
+IOUtils.closeQuietly(out);
 
 store.deleteResource(factTablePath);
-in = new FileInputStream(tmpFile);
-store.putResource(factTablePath, in, System.currentTimeMillis());
+tempIn = new FileInputStream(tmpFile);
+store.putResource(factTablePath, tempIn, 
System.currentTimeMillis());
 } finally {
 IOUtils.closeQuietly(out);
 IOUtils.closeQuietly(in);
+IOUtils.closeQuietly(tempIn);
 }
 
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/be18158d/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
--
diff --git 
a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java 
b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
index 6e5313f..bfe1d0a 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
@@ -62,10 +62,10 @@ public class BuildCubeWithStream {
 
 private static final Logger logger = 
LoggerFactory.getLogger(org.apache.kylin.provision.BuildCubeWithStream.class);
 
-private CubeManager cubeManager;
+protected CubeManager cubeManager;
 private DefaultScheduler scheduler;
 protected ExecutableManager jobService;
-private static final String cubeName = "test_streaming_table_cube";
+static final String cubeName = "test_streaming_table_cube";
 
 private KafkaConfig kafkaConfig;
 private MockKafka kafkaServer;
@@ -114,13 +114,13 @@ public class BuildCubeWithStream {
 Assert.assertEquals(topicName, topicMetadata.topic());
 }
 
-private void generateStreamData(long startTime, long endTime, int 
numberOfRecords) throws IOException {
+protected void generateStreamData(long startTime, long endTime, int 
numberOfRecords) throws IOException {
 Kafka10DataLoader dataLoader = new Kafka10DataLoader(kafkaConfig);
 DeployUtil.prepareTestDataForStreamingCube(startTime, endTime, 
numberOfRecords, cubeName, dataLoader);
 logger.info("Test data inserted into Kafka");
 }
 
-private void clearSegment(String cubeName) throws Exception {
+protected void clearSegment(String cubeName) throws Exception {
 CubeInstance cube = cubeManager.getCube(cubeName);
 // remove all existing segments
 CubeUpdate cubeBuilder = new CubeUpdate(cube);
@@ -187,7 +187,7 @@ public class BuildCubeWithStream {
 return job.getId();
 }
 
-private ExecutableState buildSegment(String cubeName, long startOffset, 
long endOffset) throws Exception {
+protected ExecutableState buildSegment(String cubeName, long startOffset, 
long endOffset) throws Exception {
 CubeSegment segment = 
cubeManager.appendSegment(cubeManager.getCube(cubeName), 0, 0, startOffset, 

[20/50] [abbrv] kylin git commit: Revert "Revert "KYLIN-1762 discard job when no stream message""

2016-10-07 Thread mahongbin
Revert "Revert "KYLIN-1762 discard job when no stream message""

This reverts commit da5ba276b972f8b3c0d220252e74ac2ff73298fc.


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

Branch: refs/heads/orderedbytes
Commit: 25f8ffc0ee16b1702723d5530b22084a608fed9a
Parents: ae3d7e4
Author: shaofengshi 
Authored: Sat Sep 24 14:57:01 2016 +0800
Committer: shaofengshi 
Committed: Tue Sep 27 10:17:40 2016 +0800

--
 .../job/execution/DefaultChainedExecutable.java |  6 +++
 .../kylin/source/kafka/SeekOffsetStep.java  | 45 +++-
 2 files changed, 41 insertions(+), 10 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/25f8ffc0/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
--
diff --git 
a/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
 
b/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
index 753b389..39a5f4f 100644
--- 
a/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
+++ 
b/core-job/src/main/java/org/apache/kylin/job/execution/DefaultChainedExecutable.java
@@ -88,6 +88,7 @@ public class DefaultChainedExecutable extends 
AbstractExecutable implements Chai
 boolean allSucceed = true;
 boolean hasError = false;
 boolean hasRunning = false;
+boolean hasDiscarded = false;
 for (Executable task : jobs) {
 final ExecutableState status = task.getStatus();
 if (status == ExecutableState.ERROR) {
@@ -99,6 +100,9 @@ public class DefaultChainedExecutable extends 
AbstractExecutable implements Chai
 if (status == ExecutableState.RUNNING) {
 hasRunning = true;
 }
+if (status == ExecutableState.DISCARDED) {
+hasDiscarded = true;
+}
 }
 if (allSucceed) {
 setEndTime(System.currentTimeMillis());
@@ -110,6 +114,8 @@ public class DefaultChainedExecutable extends 
AbstractExecutable implements Chai
 notifyUserStatusChange(executableContext, 
ExecutableState.ERROR);
 } else if (hasRunning) {
 jobService.updateJobOutput(getId(), ExecutableState.RUNNING, 
null, null);
+} else if (hasDiscarded) {
+jobService.updateJobOutput(getId(), ExecutableState.DISCARDED, 
null, null);
 } else {
 jobService.updateJobOutput(getId(), ExecutableState.READY, 
null, null);
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/25f8ffc0/source-kafka/src/main/java/org/apache/kylin/source/kafka/SeekOffsetStep.java
--
diff --git 
a/source-kafka/src/main/java/org/apache/kylin/source/kafka/SeekOffsetStep.java 
b/source-kafka/src/main/java/org/apache/kylin/source/kafka/SeekOffsetStep.java
index 5dca93f..479f1b8 100644
--- 
a/source-kafka/src/main/java/org/apache/kylin/source/kafka/SeekOffsetStep.java
+++ 
b/source-kafka/src/main/java/org/apache/kylin/source/kafka/SeekOffsetStep.java
@@ -17,6 +17,10 @@
 */
 package org.apache.kylin.source.kafka;
 
+import com.google.common.base.Function;
+import com.google.common.collect.Collections2;
+import com.google.common.collect.Maps;
+import org.apache.commons.math3.util.MathUtils;
 import org.apache.kylin.source.kafka.util.KafkaClient;
 import org.apache.kylin.source.kafka.util.KafkaOffsetMapping;
 import org.apache.kafka.clients.consumer.KafkaConsumer;
@@ -34,6 +38,7 @@ import org.apache.kylin.source.kafka.config.KafkaConfig;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import javax.annotation.Nullable;
 import java.io.IOException;
 import java.util.List;
 import java.util.Map;
@@ -101,19 +106,39 @@ public class SeekOffsetStep extends AbstractExecutable {
 }
 }
 
-KafkaOffsetMapping.saveOffsetStart(segment, startOffsets);
-KafkaOffsetMapping.saveOffsetEnd(segment, endOffsets);
+long totalStartOffset = 0, totalEndOffset = 0;
+for (Long v : startOffsets.values()) {
+totalStartOffset += v;
+}
+for (Long v : endOffsets.values()) {
+totalEndOffset += v;
+}
 
-segment.setName(CubeSegment.makeSegmentName(0, 0, 
segment.getSourceOffsetStart(), segment.getSourceOffsetEnd()));
-CubeUpdate cubeBuilder = new CubeUpdate(cube);
-

[02/50] [abbrv] kylin git commit: KYLIN-2033 code review and bug fixes

2016-10-07 Thread mahongbin
KYLIN-2033 code review and bug fixes


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

Branch: refs/heads/orderedbytes
Commit: bf127a9167c506025616031374a91c57eb1355ed
Parents: afa350f
Author: Yang Li 
Authored: Sat Sep 24 14:29:39 2016 +0800
Committer: Yang Li 
Committed: Sat Sep 24 14:29:56 2016 +0800

--
 .../kylin/metadata/cachesync/Broadcaster.java   |  4 ++--
 .../realization/RealizationRegistry.java|  8 
 .../kylin/rest/controller/CacheController.java  | 17 +---
 .../apache/kylin/rest/service/AdminService.java |  1 -
 .../apache/kylin/rest/service/CacheService.java | 21 
 .../apache/kylin/rest/service/QueryService.java |  4 +---
 webapp/app/js/services/cache.js |  2 +-
 7 files changed, 39 insertions(+), 18 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/bf127a91/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
--
diff --git 
a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
 
b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
index 380dac8..75b2333 100644
--- 
a/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
+++ 
b/core-metadata/src/main/java/org/apache/kylin/metadata/cachesync/Broadcaster.java
@@ -112,7 +112,7 @@ public class Broadcaster {
 while (true) {
 try {
 final BroadcastEvent broadcastEvent = 
broadcastEvents.takeFirst();
-logger.info("Announcing new broadcast event:" + 
broadcastEvent);
+logger.info("Announcing new broadcast event: " + 
broadcastEvent);
 for (final RestClient restClient : restClients) {
 wipingCachePool.execute(new Runnable() {
 @Override
@@ -326,7 +326,7 @@ public class Broadcaster {
 
 @Override
 public String toString() {
-return Objects.toStringHelper(this).add("type", 
entity).add("name", cacheKey).add("action", event).toString();
+return Objects.toStringHelper(this).add("entity", 
entity).add("event", event).add("cacheKey", cacheKey).toString();
 }
 
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/bf127a91/core-metadata/src/main/java/org/apache/kylin/metadata/realization/RealizationRegistry.java
--
diff --git 
a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/RealizationRegistry.java
 
b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/RealizationRegistry.java
index e6b1a86..77e2679 100644
--- 
a/core-metadata/src/main/java/org/apache/kylin/metadata/realization/RealizationRegistry.java
+++ 
b/core-metadata/src/main/java/org/apache/kylin/metadata/realization/RealizationRegistry.java
@@ -26,6 +26,7 @@ import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.util.ClassUtil;
+import org.apache.kylin.metadata.cachesync.Broadcaster;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -75,6 +76,13 @@ public class RealizationRegistry {
 logger.info("Initializing RealizationRegistry with metadata url " + 
config);
 this.config = config;
 init();
+
+Broadcaster.getInstance(config).registerListener(new 
Broadcaster.Listener() {
+@Override
+public void onClearAll(Broadcaster broadcaster) throws IOException 
{
+clearCache();
+}
+}, "");
 }
 
 private void init() {

http://git-wip-us.apache.org/repos/asf/kylin/blob/bf127a91/server-base/src/main/java/org/apache/kylin/rest/controller/CacheController.java
--
diff --git 
a/server-base/src/main/java/org/apache/kylin/rest/controller/CacheController.java
 
b/server-base/src/main/java/org/apache/kylin/rest/controller/CacheController.java
index 667046b..254aabf 100644
--- 
a/server-base/src/main/java/org/apache/kylin/rest/controller/CacheController.java
+++ 
b/server-base/src/main/java/org/apache/kylin/rest/controller/CacheController.java
@@ -47,13 +47,16 @@ public class CacheController extends BasicController {
 private CacheService cacheService;
 
 /**
- * Wipe system cache
- *
- * @param entity  {@link Broadcaster.TYPE}
- * @param event {@link Broadcaster.Event}
-   

[17/50] [abbrv] kylin git commit: Revert "Revert "rename the streaming_table.json""

2016-10-07 Thread mahongbin
Revert "Revert "rename the streaming_table.json""

This reverts commit bec25b4638835301a102141e56e12cd38d09139f.


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

Branch: refs/heads/orderedbytes
Commit: 8136380f0aa6b0594b2882952e36077c6e474a3c
Parents: 25f8ffc
Author: shaofengshi 
Authored: Sat Sep 24 14:57:11 2016 +0800
Committer: shaofengshi 
Committed: Tue Sep 27 10:17:40 2016 +0800

--
 .../kafka/DEFAULT.STREAMING_TABLE.json  | 21 
 .../streaming/DEFAULT.STREAMING_TABLE.json  |  6 --
 2 files changed, 27 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/8136380f/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json
--
diff --git 
a/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json 
b/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json
deleted file mode 100644
index 6a64cce..000
--- a/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json
+++ /dev/null
@@ -1,21 +0,0 @@
-{
- 
-  "uuid": "8b2b9dfe-777c-4d39-bf89-8472ec909193",
-  "name": "DEFAULT.STREAMING_TABLE",
-  "topic": "test_streaming_table_topic_xyz",
-  "timeout": 6,
-  "bufferSize": 65536,
-  "parserName": "org.apache.kylin.source.kafka.TimedJsonStreamParser",
-  "last_modified": 0,
-  "clusters": [
-{
-  "brokers": [
-{
-  "id": 0,
-  "host": "sandbox",
-  "port": 6667
-}
-  ]
-}
-  ]
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/8136380f/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json
--
diff --git 
a/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json 
b/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json
deleted file mode 100644
index 85a477b..000
--- a/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json
+++ /dev/null
@@ -1,6 +0,0 @@
-{
-  "uuid": "8b2b9dfe-777c-4d39-bf89-8472ec909193",
-  "name": "DEFAULT.STREAMING_TABLE",
-  "type": "kafka",
-  "last_modified": 0
-}



[06/50] [abbrv] kylin git commit: KYLIN-1999: Fix UT for the config update

2016-10-07 Thread mahongbin
KYLIN-1999: Fix UT for the config update

Signed-off-by: shaofengshi 


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

Branch: refs/heads/orderedbytes
Commit: b95e9671042240761ec8667d8ba0ae80227869e2
Parents: 61a3e7e
Author: Yiming Liu 
Authored: Sun Sep 25 17:05:02 2016 +0800
Committer: shaofengshi 
Committed: Sun Sep 25 17:07:48 2016 +0800

--
 .../test/java/org/apache/kylin/cube/CubeSpecificConfigTest.java| 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/b95e9671/core-cube/src/test/java/org/apache/kylin/cube/CubeSpecificConfigTest.java
--
diff --git 
a/core-cube/src/test/java/org/apache/kylin/cube/CubeSpecificConfigTest.java 
b/core-cube/src/test/java/org/apache/kylin/cube/CubeSpecificConfigTest.java
index dd52f82..c61f07f 100644
--- a/core-cube/src/test/java/org/apache/kylin/cube/CubeSpecificConfigTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/cube/CubeSpecificConfigTest.java
@@ -54,7 +54,7 @@ public class CubeSpecificConfigTest extends 
LocalFileMetadataTestCase {
 }
 
 private void verifyOverride(KylinConfig base, KylinConfig override) {
-assertEquals("none", base.getHbaseDefaultCompressionCodec());
+assertEquals("snappy", base.getHbaseDefaultCompressionCodec());
 assertEquals("lz4", override.getHbaseDefaultCompressionCodec());
 }
 }



[47/50] [abbrv] kylin git commit: minor, bin/sample.sh

2016-10-07 Thread mahongbin
minor, bin/sample.sh


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

Branch: refs/heads/orderedbytes
Commit: cd9de01c690193efede9114a6a31d1a964b0a29f
Parents: 95ebc85
Author: Yang Li 
Authored: Thu Oct 6 17:02:35 2016 +0800
Committer: Yang Li 
Committed: Thu Oct 6 17:03:13 2016 +0800

--
 build/bin/sample.sh | 6 --
 1 file changed, 4 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/cd9de01c/build/bin/sample.sh
--
diff --git a/build/bin/sample.sh b/build/bin/sample.sh
index 92caace..fdb8601 100644
--- a/build/bin/sample.sh
+++ b/build/bin/sample.sh
@@ -17,7 +17,8 @@
 # limitations under the License.
 #
 
-dir=$(dirname ${0})
+dir=$(cd -P -- "$(dirname -- "$0")" && pwd -P)
+export KYLIN_HOME=${KYLIN_HOME:-"${dir}/../"}
 
 source ${dir}/check-env.sh
 job_jar=`find -L ${KYLIN_HOME}/lib/ -name kylin-job*.jar`
@@ -57,4 +58,5 @@ sed -i "s/%default_engine_type%/${default_engine_type}/g" 
${KYLIN_HOME}/sample_c
 
 cd ${KYLIN_HOME}
 hbase org.apache.hadoop.util.RunJar ${job_jar} 
org.apache.kylin.common.persistence.ResourceTool upload 
${KYLIN_HOME}/sample_cube/metadata  || { exit 1; }
-echo "Sample cube is created successfully in project 'learn_kylin'; Restart 
Kylin server or reload the metadata from web UI to see the change."
+echo "Sample cube is created successfully in project 'learn_kylin'."
+echo "Restart Kylin server or reload the metadata from web UI to see the 
change."



[23/50] [abbrv] kylin git commit: Revert "Revert "stop scheduler on test finish""

2016-10-07 Thread mahongbin
Revert "Revert "stop scheduler on test finish""

This reverts commit e604f6527d60b767f8a46a576765cfd205b1efc5.


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

Branch: refs/heads/orderedbytes
Commit: 8cbffb40477611aa9c06bc5c22787aef83bc28d3
Parents: b1a0f4d
Author: shaofengshi 
Authored: Sat Sep 24 14:58:26 2016 +0800
Committer: shaofengshi 
Committed: Tue Sep 27 10:17:40 2016 +0800

--
 .../apache/kylin/job/impl/threadpool/BaseSchedulerTest.java| 1 +
 .../java/org/apache/kylin/provision/BuildCubeWithEngine.java   | 6 ++
 .../java/org/apache/kylin/provision/BuildCubeWithStream.java   | 1 +
 .../java/org/apache/kylin/provision/BuildCubeWithStream2.java  | 1 +
 4 files changed, 9 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/8cbffb40/core-job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java
--
diff --git 
a/core-job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java
 
b/core-job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java
index 4e092a1..97c9f8d 100644
--- 
a/core-job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java
+++ 
b/core-job/src/test/java/org/apache/kylin/job/impl/threadpool/BaseSchedulerTest.java
@@ -55,6 +55,7 @@ public abstract class BaseSchedulerTest extends 
LocalFileMetadataTestCase {
 
 @After
 public void after() throws Exception {
+DefaultScheduler.destroyInstance();
 cleanupTestMetadata();
 }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/8cbffb40/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
--
diff --git 
a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java 
b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
index 3d60a3c..31cf0eb 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithEngine.java
@@ -79,6 +79,7 @@ public class BuildCubeWithEngine {
 BuildCubeWithEngine buildCubeWithEngine = new 
BuildCubeWithEngine();
 buildCubeWithEngine.before();
 buildCubeWithEngine.build();
+buildCubeWithEngine.after();
 logger.info("Build is done");
 afterClass();
 logger.info("Going to exit");
@@ -148,6 +149,11 @@ public class BuildCubeWithEngine {
 
 }
 
+
+public void after(){
+DefaultScheduler.destroyInstance();
+}
+
 public static void afterClass() {
 HBaseMetadataTestCase.staticCleanupTestMetadata();
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/8cbffb40/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
--
diff --git 
a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java 
b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
index b7c609e..6e5313f 100644
--- a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
+++ b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream.java
@@ -217,6 +217,7 @@ public class BuildCubeWithStream {
 
 public void after() {
 kafkaServer.stop();
+DefaultScheduler.destroyInstance();
 }
 
 protected void waitForJob(String jobId) {

http://git-wip-us.apache.org/repos/asf/kylin/blob/8cbffb40/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream2.java
--
diff --git 
a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream2.java 
b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream2.java
index d48a473..2812446 100644
--- 
a/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream2.java
+++ 
b/kylin-it/src/test/java/org/apache/kylin/provision/BuildCubeWithStream2.java
@@ -235,6 +235,7 @@ public class BuildCubeWithStream2 {
 
 public void after() {
 kafkaServer.stop();
+DefaultScheduler.destroyInstance();
 }
 
 protected void waitForJob(String jobId) {



[40/50] [abbrv] kylin git commit: minor, fix sample cube desc

2016-10-07 Thread mahongbin
minor, fix sample cube desc


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

Branch: refs/heads/orderedbytes
Commit: 92fc0e53845a30a320467e3657a90343d2002e8c
Parents: 3a6b1f2
Author: Yang Li 
Authored: Fri Sep 30 21:46:31 2016 +0800
Committer: Yang Li 
Committed: Fri Sep 30 21:48:35 2016 +0800

--
 .../org/apache/kylin/engine/mr/HadoopUtil.java  |  3 +-
 .../cube_desc/kylin_sales_cube_desc.json| 46 ++--
 2 files changed, 24 insertions(+), 25 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/92fc0e53/engine-mr/src/main/java/org/apache/kylin/engine/mr/HadoopUtil.java
--
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/HadoopUtil.java 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/HadoopUtil.java
index 0ed39d5..88692a0 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/HadoopUtil.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/HadoopUtil.java
@@ -34,6 +34,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 public class HadoopUtil {
+@SuppressWarnings("unused")
 private static final Logger logger = 
LoggerFactory.getLogger(HadoopUtil.class);
 private static final ThreadLocal hadoopConfig = new 
ThreadLocal<>();
 
@@ -45,11 +46,9 @@ public class HadoopUtil {
 if (hadoopConfig.get() == null) {
 Configuration conf = healSickConfig(new Configuration());
 // do not cache this conf, or will affect following mr jobs
-logger.info("The conf for current mapper will be " + 
System.identityHashCode(conf));
 return conf;
 }
 Configuration conf = hadoopConfig.get();
-logger.info("The conf for current mapper will be " + 
System.identityHashCode(conf));
 return conf;
 }
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/92fc0e53/examples/sample_cube/template/cube_desc/kylin_sales_cube_desc.json
--
diff --git a/examples/sample_cube/template/cube_desc/kylin_sales_cube_desc.json 
b/examples/sample_cube/template/cube_desc/kylin_sales_cube_desc.json
index 70b1294..99f8f46 100644
--- a/examples/sample_cube/template/cube_desc/kylin_sales_cube_desc.json
+++ b/examples/sample_cube/template/cube_desc/kylin_sales_cube_desc.json
@@ -1,7 +1,10 @@
 {
   "uuid" : "0ef9b7a8-3929-4dff-b59d-2100aadc8dbf",
+  "last_modified" : 1451468470824,
   "name" : "kylin_sales_cube_desc",
+  "model_name" : "kylin_sales_model",
   "description" : null,
+  "null_string" : null,
   "dimensions" : [ {
 "name" : "CAL_DT",
 "table" : "DEFAULT.KYLIN_CAL_DT",
@@ -106,21 +109,21 @@
 },
 "dependent_measure_ref" : null
   }, {
-"name": "TOP_SELLER",
-"function": {
-  "expression": "TOP_N",
-  "parameter": {
-"type": "column",
-"value": "PRICE",
-"next_parameter": {
-  "type": "column",
-  "value": "SELLER_ID",
-  "next_parameter": null
+"name" : "TOP_SELLER",
+"function" : {
+  "expression" : "TOP_N",
+  "parameter" : {
+"type" : "column",
+"value" : "PRICE",
+"next_parameter" : {
+  "type" : "column",
+  "value" : "SELLER_ID",
+  "next_parameter" : null
 }
   },
-  "returntype": "topn(100)"
+  "returntype" : "topn(100)"
 },
-"dependent_measure_ref": null
+"dependent_measure_ref" : null
   } ],
   "rowkey" : {
 "rowkey_columns" : [ {
@@ -146,21 +149,18 @@
   "encoding" : "dict"
 } ]
   },
-  "last_modified" : 1451468470824,
-  "model_name" : "kylin_sales_model",
-  "null_string" : null,
   "hbase_mapping" : {
 "column_family" : [ {
-  "name" : "f1",
+  "name" : "F1",
   "columns" : [ {
-"qualifier" : "m",
-"measure_refs" : [ "GMV_SUM", "GMV_MIN", "GMV_MAX", "TRANS_CNT" ]
+"qualifier" : "M",
+"measure_refs" : [ "GMV_SUM", "GMV_MIN", "GMV_MAX", "TRANS_CNT", 
"TOP_SELLER" ]
   } ]
 }, {
-  "name" : "f2",
+  "name" : "F2",
   "columns" : [ {
-"qualifier" : "m",
-"measure_refs" : [ "SELLER_CNT_HLL", "SELLER_FORMAT_CNT", "TOP_SELLER" 
]
+"qualifier" : "M",
+"measure_refs" : [ "SELLER_CNT_HLL", "SELLER_FORMAT_CNT" ]
   } ]
 } ]
   },
@@ -174,10 +174,10 @@
   } ],
   "notify_list" : null,
   "status_need_notify" : [ ],
+  "partition_date_start" : 132537600,
   "auto_merge_time_ranges" : null,
   "retention_range" : 0,
   "engine_type" : %default_engine_type%,
 

[09/50] [abbrv] kylin git commit: KYLIN-1995: Upgrade deprecated properties for Hadoop 2.6

2016-10-07 Thread mahongbin
KYLIN-1995: Upgrade deprecated properties for Hadoop 2.6


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

Branch: refs/heads/orderedbytes
Commit: 2993e7bdd8875e6b2f8893d7852bb67d226f6c9e
Parents: e05cb60
Author: Yiming Liu 
Authored: Sun Sep 4 17:31:45 2016 +0800
Committer: Yang Li 
Committed: Mon Sep 26 19:51:04 2016 +0800

--
 build/conf/kylin_hive_conf.xml  |  2 +-
 build/conf/kylin_job_conf.xml   | 12 ++--
 build/conf/kylin_job_conf_inmem.xml | 12 ++--
 3 files changed, 13 insertions(+), 13 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/2993e7bd/build/conf/kylin_hive_conf.xml
--
diff --git a/build/conf/kylin_hive_conf.xml b/build/conf/kylin_hive_conf.xml
index 30c4feb..4a1b3b6 100644
--- a/build/conf/kylin_hive_conf.xml
+++ b/build/conf/kylin_hive_conf.xml
@@ -62,7 +62,7 @@
 
 -->
 
-mapred.output.compression.type
+mapreduce.output.fileoutputformat.compress.type
 BLOCK
 The compression type to use for job outputs
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/2993e7bd/build/conf/kylin_job_conf.xml
--
diff --git a/build/conf/kylin_job_conf.xml b/build/conf/kylin_job_conf.xml
index 96b806c..17a9145 100644
--- a/build/conf/kylin_job_conf.xml
+++ b/build/conf/kylin_job_conf.xml
@@ -26,7 +26,7 @@
 
 
 
-mapred.compress.map.output
+mapreduce.map.output.compress
 true
 Compress map outputs
 
@@ -37,14 +37,14 @@
 -->
 
 
-mapred.output.compress
+mapreduce.output.fileoutputformat.compress
 true
 Compress the output of a MapReduce job
 
@@ -54,14 +54,14 @@
 -->
 
 
-mapred.output.compression.type
+mapreduce.output.fileoutputformat.compress.type
 BLOCK
 The compression type to use for job outputs
 
@@ -80,7 +80,7 @@
 
 
 
-mapred.task.timeout
+mapreduce.task.timeout
 360
 Set task timeout to 1 hour
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/2993e7bd/build/conf/kylin_job_conf_inmem.xml
--
diff --git a/build/conf/kylin_job_conf_inmem.xml 
b/build/conf/kylin_job_conf_inmem.xml
index fea2f68..1d349f6 100644
--- a/build/conf/kylin_job_conf_inmem.xml
+++ b/build/conf/kylin_job_conf_inmem.xml
@@ -26,7 +26,7 @@
 
 
 
-mapred.compress.map.output
+mapreduce.map.output.compress
 true
 Compress map outputs
 
@@ -37,14 +37,14 @@
 -->
 
 
-mapred.output.compress
+mapreduce.output.fileoutputformat.compress
 true
 Compress the output of a MapReduce job
 
@@ -54,14 +54,14 @@
 -->
 
 
-mapred.output.compression.type
+mapreduce.output.fileoutputformat.compress.type
 BLOCK
 The compression type to use for job outputs
 
@@ -80,7 +80,7 @@
 
 
 
-mapred.task.timeout
+mapreduce.task.timeout
 360
 Set task timeout to 1 hour
 



[36/50] [abbrv] kylin git commit: KYLIN-1962: Revert unsed split config feature

2016-10-07 Thread mahongbin
KYLIN-1962: Revert unsed split config feature

Signed-off-by: shaofengshi 


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

Branch: refs/heads/orderedbytes
Commit: 51b5b31bbce3aac1d3a3c0de7746334961d5ffcd
Parents: 0fd4ad5
Author: Yiming Liu 
Authored: Tue Sep 27 07:52:52 2016 +0800
Committer: shaofengshi 
Committed: Wed Sep 28 17:51:49 2016 +0800

--
 .../org/apache/kylin/common/KylinConfig.java| 45 ---
 .../apache/kylin/common/KylinConfigTest.java| 12 
 .../test_case_data/localmeta/kylin.properties   | 33 ++-
 .../localmeta/kylin_account.properties  | 59 
 .../org/apache/kylin/tool/DiagnosisInfoCLI.java |  4 +-
 5 files changed, 33 insertions(+), 120 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/51b5b31b/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 f134ad4..9543fc7 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
@@ -47,7 +47,6 @@ public class KylinConfig extends KylinConfigBase {
 
 /** Kylin properties file name */
 public static final String KYLIN_CONF_PROPERTIES_FILE = "kylin.properties";
-public static final String KYLIN_ACCOUNT_CONF_PROPERTIES_FILE = 
"kylin_account.properties";
 public static final String KYLIN_CONF = "KYLIN_CONF";
 
 // static cached instances
@@ -205,23 +204,6 @@ public class KylinConfig extends KylinConfigBase {
 return getKylinPropertiesFile(path);
 }
 
-static File getKylinAccountPropertiesFile() {
-String kylinConfHome = System.getProperty(KYLIN_CONF);
-if (!StringUtils.isEmpty(kylinConfHome)) {
-logger.info("Use KYLIN_CONF=" + kylinConfHome);
-return getKylinAccountPropertiesFile(kylinConfHome);
-}
-
-logger.warn("KYLIN_CONF property was not set, will seek KYLIN_HOME env 
variable");
-
-String kylinHome = getKylinHome();
-if (StringUtils.isEmpty(kylinHome))
-throw new KylinConfigCannotInitException("Didn't find KYLIN_CONF 
or KYLIN_HOME, please set one of them");
-
-String path = kylinHome + File.separator + "conf";
-return getKylinAccountPropertiesFile(path);
-}
-
 public static Properties getKylinProperties() {
 File propFile = getKylinPropertiesFile();
 if (propFile == null || !propFile.exists()) {
@@ -242,25 +224,6 @@ public class KylinConfig extends KylinConfigBase {
 IOUtils.closeQuietly(ois);
 conf.putAll(propOverride);
 }
-
-File accountPropFile = getKylinAccountPropertiesFile();
-if (accountPropFile.exists()) {
-FileInputStream ois = new FileInputStream(accountPropFile);
-Properties propAccount = new Properties();
-propAccount.load(ois);
-IOUtils.closeQuietly(ois);
-conf.putAll(propAccount);
-}
-
-File accountPropOverrideFile = new 
File(accountPropFile.getParentFile(), accountPropFile.getName() + ".override");
-if (accountPropOverrideFile.exists()) {
-FileInputStream ois = new 
FileInputStream(accountPropOverrideFile);
-Properties propAccountOverride = new Properties();
-propAccountOverride.load(ois);
-IOUtils.closeQuietly(ois);
-conf.putAll(propAccountOverride);
-}
-
 } catch (IOException e) {
 throw new RuntimeException(e);
 }
@@ -282,14 +245,6 @@ public class KylinConfig extends KylinConfigBase {
 return new File(path, KYLIN_CONF_PROPERTIES_FILE);
 }
 
-private static File getKylinAccountPropertiesFile(String path) {
-if (path == null) {
-return null;
-}
-
-return new File(path, KYLIN_ACCOUNT_CONF_PROPERTIES_FILE);
-}
-
 public static void setSandboxEnvIfPossible() {
 File dir1 = new File("../examples/test_case_data/sandbox");
 File dir2 = new File("../../kylin/examples/test_case_data/sandbox");

http://git-wip-us.apache.org/repos/asf/kylin/blob/51b5b31b/core-common/src/test/java/org/apache/kylin/common/KylinConfigTest.java

kylin git commit: improve integer type upgrade logic

2016-09-30 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/orderedbytes [created] 4aa8ede2c


improve integer type upgrade logic


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

Branch: refs/heads/orderedbytes
Commit: 4aa8ede2c1a34254c8bc0752ec79abc3c8d47ce4
Parents: fa5c9cb
Author: Hongbin Ma 
Authored: Fri Sep 30 18:31:04 2016 +0800
Committer: Hongbin Ma 
Committed: Fri Sep 30 18:31:04 2016 +0800

--
 .../filter/EvaluatableFunctionTupleFilter.java  | 151 ---
 .../metadata/filter/TupleFilterSerializer.java  |  20 ++-
 .../apache/kylin/metadata/model/ColumnDesc.java |  43 +++---
 .../apache/kylin/metadata/tuple/TupleInfo.java  |   4 +-
 .../apache/kylin/query/schema/OLAPTable.java|  31 +++-
 5 files changed, 72 insertions(+), 177 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/4aa8ede2/core-metadata/src/main/java/org/apache/kylin/metadata/filter/EvaluatableFunctionTupleFilter.java
--
diff --git 
a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/EvaluatableFunctionTupleFilter.java
 
b/core-metadata/src/main/java/org/apache/kylin/metadata/filter/EvaluatableFunctionTupleFilter.java
deleted file mode 100644
index ff24172..000
--- 
a/core-metadata/src/main/java/org/apache/kylin/metadata/filter/EvaluatableFunctionTupleFilter.java
+++ /dev/null
@@ -1,151 +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.
- */
-
-package org.apache.kylin.metadata.filter;
-
-import java.lang.reflect.InvocationTargetException;
-import java.nio.ByteBuffer;
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.datatype.DataType;
-import org.apache.kylin.metadata.datatype.StringSerializer;
-import org.apache.kylin.metadata.model.TblColRef;
-import org.apache.kylin.metadata.tuple.IEvaluatableTuple;
-
-import com.google.common.collect.Lists;
-
-public class EvaluatableFunctionTupleFilter extends BuiltInFunctionTupleFilter 
{
-
-private boolean constantsInitted = false;
-
-//about non-like
-private List values;
-private Object tupleValue;
-
-public EvaluatableFunctionTupleFilter(String name) {
-super(name, FilterOperatorEnum.EVAL_FUNC);
-values = Lists.newArrayListWithCapacity(1);
-values.add(null);
-}
-
-@Override
-public boolean evaluate(IEvaluatableTuple tuple, IFilterCodeSystem cs) {
-
-// extract tuple value
-Object tupleValue = null;
-for (TupleFilter filter : this.children) {
-if (!isConstant(filter)) {
-filter.evaluate(tuple, cs);
-tupleValue = filter.getValues().iterator().next();
-}
-}
-
-TblColRef tblColRef = this.getColumn();
-DataType strDataType = DataType.getType("string");
-if (tblColRef.getType() != strDataType) {
-throw new IllegalStateException("Only String type is allow in 
BuiltInFunction");
-}
-ByteArray valueByteArray = (ByteArray) tupleValue;
-StringSerializer serializer = new StringSerializer(strDataType);
-String value = 
serializer.deserialize(ByteBuffer.wrap(valueByteArray.array(), 
valueByteArray.offset(), valueByteArray.length()));
-
-try {
-if (isLikeFunction()) {
-return (Boolean) invokeFunction(value);
-} else {
-this.tupleValue = invokeFunction(value);
-//convert back to ByteArray format because the outer 
EvaluatableFunctionTupleFilter assumes input as ByteArray
-ByteBuffer buffer = 
ByteBuffer.allocate(valueByteArray.length() * 2);
-serializer.serialize((String) this.tupleValue, buffer);
-  

kylin git commit: add beeline doc

2016-09-28 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/document 1ec5aca9a -> 7a97d513c


add beeline doc


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

Branch: refs/heads/document
Commit: 7a97d513c41ba865aa9acb0872ae0cf299353439
Parents: 1ec5aca
Author: Hongbin Ma 
Authored: Thu Sep 29 13:24:23 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Sep 29 13:24:23 2016 +0800

--
 website/_docs15/howto/howto_use_beeline.md | 14 ++
 1 file changed, 14 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/7a97d513/website/_docs15/howto/howto_use_beeline.md
--
diff --git a/website/_docs15/howto/howto_use_beeline.md 
b/website/_docs15/howto/howto_use_beeline.md
new file mode 100644
index 000..f0aea67
--- /dev/null
+++ b/website/_docs15/howto/howto_use_beeline.md
@@ -0,0 +1,14 @@
+---
+layout: docs15
+title:  Use Beeline for Hive Commands
+categories: howto
+permalink: /docs15/howto/howto_use_beeline.html
+---
+
+Beeline(https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients) 
is recommended by many venders to replace Hive CLI. By default Kylin uses Hive 
CLI to synchronize Hive tables, create flatten intermediate tables, etc. By 
simple configuration changes you can set Kylin to use Beeline instead.
+
+Edit $KYLIN_HOME/conf/kylin.properties by:
+
+  1. change kylin.hive.client=cli to kylin.hive.client=beeline
+  2. add "kylin.hive.beeline.params", this is where you can specifiy beeline 
commmand parameters. Like username(-n), JDBC URL(-u),etc. There's a sample 
kylin.hive.beeline.params included in default kylin.properties, however it's 
commented. You can modify the sample based on your real environment.
+



kylin git commit: add beeline doc

2016-09-28 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/document 649d382d5 -> 1ec5aca9a


add beeline doc


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

Branch: refs/heads/document
Commit: 1ec5aca9a3c878872c5754992fbd3f71272d83e8
Parents: 649d382
Author: Hongbin Ma 
Authored: Thu Sep 29 13:16:52 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Sep 29 13:16:52 2016 +0800

--
 website/_data/docs15.yml | 1 +
 1 file changed, 1 insertion(+)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/1ec5aca9/website/_data/docs15.yml
--
diff --git a/website/_data/docs15.yml b/website/_data/docs15.yml
index 6aa78b2..25be101 100644
--- a/website/_data/docs15.yml
+++ b/website/_data/docs15.yml
@@ -57,4 +57,5 @@
   - howto/howto_jdbc
   - howto/howto_upgrade
   - howto/howto_ldap_and_sso
+  - howto/howto_use_beeline
   - howto/howto_update_coprocessor



kylin git commit: minor changes

2016-09-21 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/master 32a9c6706 -> 2359a75e3


minor changes


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

Branch: refs/heads/master
Commit: 2359a75e3ca9d1a9c5f06331ab02edce1385b66d
Parents: 32a9c67
Author: Hongbin Ma 
Authored: Thu Sep 22 10:45:08 2016 +0800
Committer: Hongbin Ma 
Committed: Thu Sep 22 10:45:15 2016 +0800

--
 .../java/org/apache/kylin/metadata/datatype/BooleanSerializer.java | 2 +-
 .../kylin/storage/gtrecord/SortedIteratorMergerWithLimit.java  | 1 +
 .../java/org/apache/kylin/rest/controller/QueryController.java | 2 +-
 3 files changed, 3 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/2359a75e/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BooleanSerializer.java
--
diff --git 
a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BooleanSerializer.java
 
b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BooleanSerializer.java
index e843247..acb6de1 100644
--- 
a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BooleanSerializer.java
+++ 
b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BooleanSerializer.java
@@ -25,7 +25,7 @@ import org.apache.commons.lang.BooleanUtils;
 
 public class BooleanSerializer extends DataTypeSerializer {
 
-final String[] TRUE_VALUE_SET = { "true", "t", "on", "yes" };
+public final static String[] TRUE_VALUE_SET = { "true", "t", "on", "yes" };
 
 // be thread-safe and avoid repeated obj creation
 private ThreadLocal current = new ThreadLocal();

http://git-wip-us.apache.org/repos/asf/kylin/blob/2359a75e/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SortedIteratorMergerWithLimit.java
--
diff --git 
a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SortedIteratorMergerWithLimit.java
 
b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SortedIteratorMergerWithLimit.java
index 0e40150..2f2691e 100644
--- 
a/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SortedIteratorMergerWithLimit.java
+++ 
b/core-storage/src/main/java/org/apache/kylin/storage/gtrecord/SortedIteratorMergerWithLimit.java
@@ -94,6 +94,7 @@ public class SortedIteratorMergerWithLimit extends SortedIt
 PeekingImpl first = heap.poll();
 E current = first.next();
 try {
+//clone is protected on Object, have to use reflection to 
call the overwritten clone method in subclasses
 current = (E) 
current.getClass().getMethod("clone").invoke(current);
 } catch (IllegalAccessException | InvocationTargetException | 
NoSuchMethodException e) {
 throw new RuntimeException(e);

http://git-wip-us.apache.org/repos/asf/kylin/blob/2359a75e/server-base/src/main/java/org/apache/kylin/rest/controller/QueryController.java
--
diff --git 
a/server-base/src/main/java/org/apache/kylin/rest/controller/QueryController.java
 
b/server-base/src/main/java/org/apache/kylin/rest/controller/QueryController.java
index 5cf6492..62cb0a7 100644
--- 
a/server-base/src/main/java/org/apache/kylin/rest/controller/QueryController.java
+++ 
b/server-base/src/main/java/org/apache/kylin/rest/controller/QueryController.java
@@ -205,7 +205,7 @@ public class QueryController extends BasicController {
 checkQueryAuth(sqlResponse);
 
 } catch (Throwable e) { // calcite may throw AssertError
-//logger.error("Exception when execute sql", e);
+logger.error("Exception when execute sql", e);
 String errMsg = QueryUtil.makeErrorMsgUserFriendly(e);
 
 sqlResponse = new SQLResponse(null, null, 0, true, errMsg);



[2/3] kylin git commit: KYLIN-2030 query on lookup table with no derived

2016-09-20 Thread mahongbin
KYLIN-2030 query on lookup table with no derived


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

Branch: refs/heads/master
Commit: dbc02b75b09e07dbeee7bd417a29e5e9a01a0adf
Parents: 472e6d5
Author: Hongbin Ma 
Authored: Tue Sep 20 18:45:36 2016 +0800
Committer: Hongbin Ma 
Committed: Tue Sep 20 21:17:12 2016 +0800

--
 .../kylin/cube/CubeCapabilityChecker.java   | 24 +---
 1 file changed, 21 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/dbc02b75/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
--
diff --git 
a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java 
b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
index caef529..1eada16 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
@@ -25,14 +25,18 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.measure.MeasureType;
 import org.apache.kylin.measure.basic.BasicMeasureType;
+import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.filter.UDF.MassInTupleFilter;
+import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.IStorageAware;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.ParameterDesc;
+import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.CapabilityResult;
 import org.apache.kylin.metadata.realization.SQLDigest;
@@ -73,7 +77,21 @@ public class CubeCapabilityChecker {
 
 // try dimension-as-measure
 if (!unmatchedAggregations.isEmpty()) {
-tryDimensionAsMeasures(unmatchedAggregations, digest, cube, 
result);
+if (cube.getDescriptor().getFactTable().equals(digest.factTable)) {
+tryDimensionAsMeasures(unmatchedAggregations, digest, cube, 
result, cube.getDescriptor().listDimensionColumnsIncludingDerived());
+} else {
+//deal with query on lookup table, like 
https://issues.apache.org/jira/browse/KYLIN-2030
+if 
(cube.getSegments().get(0).getSnapshots().containsKey(digest.factTable)) {
+TableDesc tableDesc = 
MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getTableDesc(digest.factTable);
+Set dimCols = Sets.newHashSet();
+for (ColumnDesc columnDesc : tableDesc.getColumns()) {
+dimCols.add(columnDesc.getRef());
+}
+tryDimensionAsMeasures(unmatchedAggregations, digest, 
cube, result, dimCols);
+} else {
+logger.info("Skip tryDimensionAsMeasures because current 
cube {} does not touch lookup table {} at all", cube.getName(), 
digest.factTable);
+}
+}
 }
 
 if (!unmatchedDimensions.isEmpty()) {
@@ -129,7 +147,7 @@ public class CubeCapabilityChecker {
 return result;
 }
 
-private static void tryDimensionAsMeasures(Collection 
unmatchedAggregations, SQLDigest digest, CubeInstance cube, CapabilityResult 
result) {
+private static void tryDimensionAsMeasures(Collection 
unmatchedAggregations, SQLDigest digest, CubeInstance cube, CapabilityResult 
result, Set dimCols) {
 CubeDesc cubeDesc = cube.getDescriptor();
 Collection cubeFuncs = cubeDesc.listAllFunctions();
 
@@ -154,7 +172,7 @@ public class CubeCapabilityChecker {
 continue;
 }
 List neededCols = parameterDesc.getColRefs();
-if (neededCols.size() > 0 && 
cubeDesc.listDimensionColumnsIncludingDerived().containsAll(neededCols) && 
FunctionDesc.BUILT_IN_AGGREGATIONS.contains(functionDesc.getExpression())) {
+if (neededCols.size() > 0 && dimCols.containsAll(neededCols) && 
FunctionDesc.BUILT_IN_AGGREGATIONS.contains(functionDesc.getExpression())) {
 result.influences.add(new 
CapabilityResult.DimensionAsMeasure(functionDesc));
 it.remove();
 continue;



[1/3] kylin git commit: minor, add test data provision tool for dev purpose

2016-09-20 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/master 7d716dff1 -> f62541ed4


minor, add test data provision tool for dev purpose


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

Branch: refs/heads/master
Commit: f62541ed4794adacaca508da5a4997dea82bd385
Parents: dbc02b7
Author: Hongbin Ma 
Authored: Tue Sep 20 20:46:47 2016 +0800
Committer: Hongbin Ma 
Committed: Tue Sep 20 21:17:12 2016 +0800

--
 .../kylin/provision/ProvisionWithoutBuild.java  | 29 
 1 file changed, 29 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/f62541ed/kylin-it/src/test/java/org/apache/kylin/provision/ProvisionWithoutBuild.java
--
diff --git 
a/kylin-it/src/test/java/org/apache/kylin/provision/ProvisionWithoutBuild.java 
b/kylin-it/src/test/java/org/apache/kylin/provision/ProvisionWithoutBuild.java
new file mode 100644
index 000..71bb0ca
--- /dev/null
+++ 
b/kylin-it/src/test/java/org/apache/kylin/provision/ProvisionWithoutBuild.java
@@ -0,0 +1,29 @@
+/*
+ * 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.provision;
+
+import org.apache.kylin.job.DeployUtil;
+
+public class ProvisionWithoutBuild extends BuildCubeWithEngine {
+public static void main(String[] args) throws Exception {
+beforeClass();
+DeployUtil.deployMetadata();
+
DeployUtil.prepareTestDataForNormalCubes("test_kylin_cube_with_slr_empty");
+}
+}



[3/3] kylin git commit: KYLIN-2029 distinct count query on lookup table

2016-09-20 Thread mahongbin
KYLIN-2029 distinct count query on lookup table


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

Branch: refs/heads/master
Commit: 472e6d547f2f04b8d6f92b9affa173a9685d3d2d
Parents: 7d716df
Author: Hongbin Ma 
Authored: Tue Sep 20 18:19:19 2016 +0800
Committer: Hongbin Ma 
Committed: Tue Sep 20 21:17:12 2016 +0800

--
 .../test/resources/query/sql_lookup/query05.sql  | 19 +++
 .../kylin/query/relnode/OLAPAggregateRel.java|  8 ++--
 2 files changed, 25 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/472e6d54/kylin-it/src/test/resources/query/sql_lookup/query05.sql
--
diff --git a/kylin-it/src/test/resources/query/sql_lookup/query05.sql 
b/kylin-it/src/test/resources/query/sql_lookup/query05.sql
new file mode 100644
index 000..31fdf61
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_lookup/query05.sql
@@ -0,0 +1,19 @@
+--
+-- 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.
+--
+
+select count(distinct META_CATEG_NAME) as CNT ,max(META_CATEG_NAME) as y from 
test_category_groupings

http://git-wip-us.apache.org/repos/asf/kylin/blob/472e6d54/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
--
diff --git 
a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java 
b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
index f55c86f..97efb27 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
@@ -265,7 +265,7 @@ public class OLAPAggregateRel extends Aggregate implements 
OLAPRel {
 implementor.visitChild(this, getInput());
 
 // only rewrite the innermost aggregation
-if (!this.afterAggregate && 
RewriteImplementor.needRewrite(this.context)) {
+if (!this.afterAggregate) {
 // rewrite the aggCalls
 this.rewriteAggCalls = new 
ArrayList(aggCalls.size());
 for (int i = 0; i < this.aggCalls.size(); i++) {
@@ -354,11 +354,15 @@ public class OLAPAggregateRel extends Aggregate 
implements OLAPRel {
 }
 }
 
+private boolean noPrecaculatedFieldsAvailable() {
+return !this.context.hasPrecalculatedFields() || 
!RewriteImplementor.needRewrite(this.context);
+}
+
 private AggregateCall rewriteAggregateCall(AggregateCall aggCall, 
FunctionDesc func) {
 
 //if it's not a cube, then the "needRewriteField func" should not 
resort to any rewrite fields, 
 // which do not exist at all
-if (!this.context.hasPrecalculatedFields() && func.needRewriteField()) 
{
+if (noPrecaculatedFieldsAvailable() && func.needRewriteField()) {
 logger.info(func + "skip rewriteAggregateCall because no 
pre-aggregated field available");
 return aggCall;
 }



[2/2] kylin git commit: KYLIN-2029 distinct count query on lookup table

2016-09-20 Thread mahongbin
KYLIN-2029 distinct count query on lookup table


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

Branch: refs/heads/KYLIN-2030-2
Commit: a98bea49e6255dadff6cbed7f6e47aa9187b648f
Parents: 506cd78
Author: Hongbin Ma 
Authored: Tue Sep 20 18:19:19 2016 +0800
Committer: Hongbin Ma 
Committed: Tue Sep 20 18:59:27 2016 +0800

--
 .../test/resources/query/sql_lookup/query05.sql  | 19 +++
 .../kylin/query/relnode/OLAPAggregateRel.java|  8 ++--
 2 files changed, 25 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/a98bea49/kylin-it/src/test/resources/query/sql_lookup/query05.sql
--
diff --git a/kylin-it/src/test/resources/query/sql_lookup/query05.sql 
b/kylin-it/src/test/resources/query/sql_lookup/query05.sql
new file mode 100644
index 000..31fdf61
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_lookup/query05.sql
@@ -0,0 +1,19 @@
+--
+-- 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.
+--
+
+select count(distinct META_CATEG_NAME) as CNT ,max(META_CATEG_NAME) as y from 
test_category_groupings

http://git-wip-us.apache.org/repos/asf/kylin/blob/a98bea49/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
--
diff --git 
a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java 
b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
index f55c86f..97efb27 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
@@ -265,7 +265,7 @@ public class OLAPAggregateRel extends Aggregate implements 
OLAPRel {
 implementor.visitChild(this, getInput());
 
 // only rewrite the innermost aggregation
-if (!this.afterAggregate && 
RewriteImplementor.needRewrite(this.context)) {
+if (!this.afterAggregate) {
 // rewrite the aggCalls
 this.rewriteAggCalls = new 
ArrayList(aggCalls.size());
 for (int i = 0; i < this.aggCalls.size(); i++) {
@@ -354,11 +354,15 @@ public class OLAPAggregateRel extends Aggregate 
implements OLAPRel {
 }
 }
 
+private boolean noPrecaculatedFieldsAvailable() {
+return !this.context.hasPrecalculatedFields() || 
!RewriteImplementor.needRewrite(this.context);
+}
+
 private AggregateCall rewriteAggregateCall(AggregateCall aggCall, 
FunctionDesc func) {
 
 //if it's not a cube, then the "needRewriteField func" should not 
resort to any rewrite fields, 
 // which do not exist at all
-if (!this.context.hasPrecalculatedFields() && func.needRewriteField()) 
{
+if (noPrecaculatedFieldsAvailable() && func.needRewriteField()) {
 logger.info(func + "skip rewriteAggregateCall because no 
pre-aggregated field available");
 return aggCall;
 }



[1/2] kylin git commit: KYLIN-2030 query on lookup table with no derived

2016-09-20 Thread mahongbin
Repository: kylin
Updated Branches:
  refs/heads/KYLIN-2030-2 [created] 1ca903b05


KYLIN-2030 query on lookup table with no derived


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

Branch: refs/heads/KYLIN-2030-2
Commit: 1ca903b056532ec29ba51466d18091af62847877
Parents: a98bea4
Author: Hongbin Ma 
Authored: Tue Sep 20 18:45:36 2016 +0800
Committer: Hongbin Ma 
Committed: Tue Sep 20 18:59:27 2016 +0800

--
 .../kylin/cube/CubeCapabilityChecker.java   | 24 +---
 1 file changed, 21 insertions(+), 3 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/1ca903b0/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
--
diff --git 
a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java 
b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
index caef529..1eada16 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/CubeCapabilityChecker.java
@@ -25,14 +25,18 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.cube.model.CubeDesc;
 import org.apache.kylin.measure.MeasureType;
 import org.apache.kylin.measure.basic.BasicMeasureType;
+import org.apache.kylin.metadata.MetadataManager;
 import org.apache.kylin.metadata.filter.UDF.MassInTupleFilter;
+import org.apache.kylin.metadata.model.ColumnDesc;
 import org.apache.kylin.metadata.model.FunctionDesc;
 import org.apache.kylin.metadata.model.IStorageAware;
 import org.apache.kylin.metadata.model.MeasureDesc;
 import org.apache.kylin.metadata.model.ParameterDesc;
+import org.apache.kylin.metadata.model.TableDesc;
 import org.apache.kylin.metadata.model.TblColRef;
 import org.apache.kylin.metadata.realization.CapabilityResult;
 import org.apache.kylin.metadata.realization.SQLDigest;
@@ -73,7 +77,21 @@ public class CubeCapabilityChecker {
 
 // try dimension-as-measure
 if (!unmatchedAggregations.isEmpty()) {
-tryDimensionAsMeasures(unmatchedAggregations, digest, cube, 
result);
+if (cube.getDescriptor().getFactTable().equals(digest.factTable)) {
+tryDimensionAsMeasures(unmatchedAggregations, digest, cube, 
result, cube.getDescriptor().listDimensionColumnsIncludingDerived());
+} else {
+//deal with query on lookup table, like 
https://issues.apache.org/jira/browse/KYLIN-2030
+if 
(cube.getSegments().get(0).getSnapshots().containsKey(digest.factTable)) {
+TableDesc tableDesc = 
MetadataManager.getInstance(KylinConfig.getInstanceFromEnv()).getTableDesc(digest.factTable);
+Set dimCols = Sets.newHashSet();
+for (ColumnDesc columnDesc : tableDesc.getColumns()) {
+dimCols.add(columnDesc.getRef());
+}
+tryDimensionAsMeasures(unmatchedAggregations, digest, 
cube, result, dimCols);
+} else {
+logger.info("Skip tryDimensionAsMeasures because current 
cube {} does not touch lookup table {} at all", cube.getName(), 
digest.factTable);
+}
+}
 }
 
 if (!unmatchedDimensions.isEmpty()) {
@@ -129,7 +147,7 @@ public class CubeCapabilityChecker {
 return result;
 }
 
-private static void tryDimensionAsMeasures(Collection 
unmatchedAggregations, SQLDigest digest, CubeInstance cube, CapabilityResult 
result) {
+private static void tryDimensionAsMeasures(Collection 
unmatchedAggregations, SQLDigest digest, CubeInstance cube, CapabilityResult 
result, Set dimCols) {
 CubeDesc cubeDesc = cube.getDescriptor();
 Collection cubeFuncs = cubeDesc.listAllFunctions();
 
@@ -154,7 +172,7 @@ public class CubeCapabilityChecker {
 continue;
 }
 List neededCols = parameterDesc.getColRefs();
-if (neededCols.size() > 0 && 
cubeDesc.listDimensionColumnsIncludingDerived().containsAll(neededCols) && 
FunctionDesc.BUILT_IN_AGGREGATIONS.contains(functionDesc.getExpression())) {
+if (neededCols.size() > 0 && dimCols.containsAll(neededCols) && 
FunctionDesc.BUILT_IN_AGGREGATIONS.contains(functionDesc.getExpression())) {
 result.influences.add(new 
CapabilityResult.DimensionAsMeasure(functionDesc));
 it.remove();
 continue;



[2/2] kylin git commit: KYLIN-2029 distinct count query on lookup table

2016-09-20 Thread mahongbin
KYLIN-2029 distinct count query on lookup table


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

Branch: refs/heads/KYLIN-2030
Commit: d6b9650246af1b2331b694a7becda5f804fe2449
Parents: 5a049a5
Author: Hongbin Ma 
Authored: Tue Sep 20 18:19:19 2016 +0800
Committer: Hongbin Ma 
Committed: Tue Sep 20 18:45:41 2016 +0800

--
 .../test/resources/query/sql_lookup/query05.sql  | 19 +++
 .../kylin/query/relnode/OLAPAggregateRel.java|  8 ++--
 2 files changed, 25 insertions(+), 2 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/d6b96502/kylin-it/src/test/resources/query/sql_lookup/query05.sql
--
diff --git a/kylin-it/src/test/resources/query/sql_lookup/query05.sql 
b/kylin-it/src/test/resources/query/sql_lookup/query05.sql
new file mode 100644
index 000..31fdf61
--- /dev/null
+++ b/kylin-it/src/test/resources/query/sql_lookup/query05.sql
@@ -0,0 +1,19 @@
+--
+-- 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.
+--
+
+select count(distinct META_CATEG_NAME) as CNT ,max(META_CATEG_NAME) as y from 
test_category_groupings

http://git-wip-us.apache.org/repos/asf/kylin/blob/d6b96502/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
--
diff --git 
a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java 
b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
index f55c86f..97efb27 100644
--- a/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
+++ b/query/src/main/java/org/apache/kylin/query/relnode/OLAPAggregateRel.java
@@ -265,7 +265,7 @@ public class OLAPAggregateRel extends Aggregate implements 
OLAPRel {
 implementor.visitChild(this, getInput());
 
 // only rewrite the innermost aggregation
-if (!this.afterAggregate && 
RewriteImplementor.needRewrite(this.context)) {
+if (!this.afterAggregate) {
 // rewrite the aggCalls
 this.rewriteAggCalls = new 
ArrayList(aggCalls.size());
 for (int i = 0; i < this.aggCalls.size(); i++) {
@@ -354,11 +354,15 @@ public class OLAPAggregateRel extends Aggregate 
implements OLAPRel {
 }
 }
 
+private boolean noPrecaculatedFieldsAvailable() {
+return !this.context.hasPrecalculatedFields() || 
!RewriteImplementor.needRewrite(this.context);
+}
+
 private AggregateCall rewriteAggregateCall(AggregateCall aggCall, 
FunctionDesc func) {
 
 //if it's not a cube, then the "needRewriteField func" should not 
resort to any rewrite fields, 
 // which do not exist at all
-if (!this.context.hasPrecalculatedFields() && func.needRewriteField()) 
{
+if (noPrecaculatedFieldsAvailable() && func.needRewriteField()) {
 logger.info(func + "skip rewriteAggregateCall because no 
pre-aggregated field available");
 return aggCall;
 }



[09/13] kylin git commit: Revert "rename the streaming_table.json"

2016-09-19 Thread mahongbin
Revert "rename the streaming_table.json"

This reverts commit ffdc5d21d5f46daaf25a0dfed9b3afeef78fc62c.


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

Branch: refs/heads/master
Commit: bec25b4638835301a102141e56e12cd38d09139f
Parents: d1e979b
Author: Hongbin Ma 
Authored: Mon Sep 19 23:50:13 2016 +0800
Committer: Hongbin Ma 
Committed: Tue Sep 20 11:43:08 2016 +0800

--
 .../kafka/DEFAULT.STREAMING_TABLE.json  | 21 
 .../streaming/DEFAULT.STREAMING_TABLE.json  |  6 ++
 2 files changed, 27 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/bec25b46/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json
--
diff --git 
a/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json 
b/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json
new file mode 100644
index 000..6a64cce
--- /dev/null
+++ b/examples/test_case_data/localmeta/kafka/DEFAULT.STREAMING_TABLE.json
@@ -0,0 +1,21 @@
+{
+ 
+  "uuid": "8b2b9dfe-777c-4d39-bf89-8472ec909193",
+  "name": "DEFAULT.STREAMING_TABLE",
+  "topic": "test_streaming_table_topic_xyz",
+  "timeout": 6,
+  "bufferSize": 65536,
+  "parserName": "org.apache.kylin.source.kafka.TimedJsonStreamParser",
+  "last_modified": 0,
+  "clusters": [
+{
+  "brokers": [
+{
+  "id": 0,
+  "host": "sandbox",
+  "port": 6667
+}
+  ]
+}
+  ]
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/bec25b46/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json
--
diff --git 
a/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json 
b/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json
new file mode 100644
index 000..85a477b
--- /dev/null
+++ b/examples/test_case_data/localmeta/streaming/DEFAULT.STREAMING_TABLE.json
@@ -0,0 +1,6 @@
+{
+  "uuid": "8b2b9dfe-777c-4d39-bf89-8472ec909193",
+  "name": "DEFAULT.STREAMING_TABLE",
+  "type": "kafka",
+  "last_modified": 0
+}



[13/13] kylin git commit: Revert "KYLIN-1726 update to kafka 0.10"

2016-09-19 Thread mahongbin
Revert "KYLIN-1726 update to kafka 0.10"

This reverts commit 1b1b2e37fdcba7ad67f0fa3f2369aa65431f13bc.


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

Branch: refs/heads/master
Commit: 355e58ba4209ddf945663228688d550bf654c387
Parents: da5ba27
Author: Hongbin Ma 
Authored: Mon Sep 19 23:50:26 2016 +0800
Committer: Hongbin Ma 
Committed: Tue Sep 20 11:43:08 2016 +0800

--
 .../java/org/apache/kylin/cube/CubeManager.java | 30 
 .../kylin/rest/controller/CubeController.java   |  8 ++---
 .../apache/kylin/rest/service/JobService.java   |  4 +--
 .../source/kafka/util/KafkaSampleProducer.java  | 38 
 4 files changed, 27 insertions(+), 53 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/kylin/blob/355e58ba/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 11eabce..fc68798 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
@@ -679,28 +679,12 @@ public class CubeManager implements IRealizationProvider {
 return null;
 }
 
-List buildingSegs = cube.getBuildingSegments();
-if (buildingSegs.size() > 0) {
-logger.debug("Cube " + cube.getName() + " has " + 
buildingSegs.size() + " building segments");
-}
-
-List readySegs = 
cube.getSegments(SegmentStatusEnum.READY);
-
-List mergingSegs = Lists.newArrayList();
-if (buildingSegs.size() > 0) {
-
-for (CubeSegment building : buildingSegs) {
-// exclude those under-merging segs
-for (CubeSegment ready : readySegs) {
-if (ready.getSourceOffsetStart() >= 
building.getSourceOffsetStart() && ready.getSourceOffsetEnd() <= 
building.getSourceOffsetEnd()) {
-mergingSegs.add(ready);
-}
-}
-}
+if (cube.getBuildingSegments().size() > 0) {
+logger.debug("Cube " + cube.getName() + " has bulding segment, 
will not trigger merge at this moment");
+return null;
 }
 
-// exclude those already under merging segments
-readySegs.removeAll(mergingSegs);
+List ready = cube.getSegments(SegmentStatusEnum.READY);
 
 long[] timeRanges = cube.getDescriptor().getAutoMergeTimeRanges();
 Arrays.sort(timeRanges);
@@ -708,9 +692,9 @@ public class CubeManager implements IRealizationProvider {
 for (int i = timeRanges.length - 1; i >= 0; i--) {
 long toMergeRange = timeRanges[i];
 
-for (int s = 0; s < readySegs.size(); s++) {
-CubeSegment seg = readySegs.get(s);
-Pair p = 
findMergeOffsetsByDateRange(readySegs.subList(s, readySegs.size()), //
+for (int s = 0; s < ready.size(); s++) {
+CubeSegment seg = ready.get(s);
+Pair p = 
findMergeOffsetsByDateRange(ready.subList(s, ready.size()), //
 seg.getDateRangeStart(), seg.getDateRangeStart() + 
toMergeRange, toMergeRange);
 if (p != null && p.getSecond().getDateRangeEnd() - 
p.getFirst().getDateRangeStart() >= toMergeRange)
 return Pair.newPair(p.getFirst().getSourceOffsetStart(), 
p.getSecond().getSourceOffsetEnd());

http://git-wip-us.apache.org/repos/asf/kylin/blob/355e58ba/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
--
diff --git 
a/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
 
b/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
index 669f53e..42b117c 100644
--- 
a/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
+++ 
b/server-base/src/main/java/org/apache/kylin/rest/controller/CubeController.java
@@ -272,7 +272,7 @@ public class CubeController extends BasicController {
 @RequestMapping(value = "/{cubeName}/rebuild", method = { 
RequestMethod.PUT })
 @ResponseBody
 public JobInstance rebuild(@PathVariable String cubeName, @RequestBody 
JobBuildRequest req) {
-return buildInternal(cubeName, req.getStartTime(), req.getEndTime(), 
0, 0, req.getBuildType(), true, req.isForce() 

<    1   2   3   4   5   6   7   8   9   10   >