You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by sh...@apache.org on 2017/01/05 07:21:03 UTC
[01/17] kylin git commit: KYLIN-2322 fix TictionaryDictionaryForest
cache bug and add CacheDictionary interface [Forced Update!]
Repository: kylin
Updated Branches:
refs/heads/sparkcubing-rebase a9c8b61bd -> afd1ac223 (forced update)
KYLIN-2322 fix TictionaryDictionaryForest cache bug and add CacheDictionary interface
Signed-off-by: Li Yang <li...@apache.org>
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/1d6a36bf
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/1d6a36bf
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/1d6a36bf
Branch: refs/heads/sparkcubing-rebase
Commit: 1d6a36bf66085e0bed79876379b21677ec55788f
Parents: d7971d9
Author: xiefan46 <95...@qq.com>
Authored: Wed Dec 28 18:10:07 2016 +0800
Committer: Li Yang <li...@apache.org>
Committed: Tue Jan 3 15:43:24 2017 +0800
----------------------------------------------------------------------
.../apache/kylin/common/KylinConfigBase.java | 2 +-
.../apache/kylin/common/util/Dictionary.java | 2 +
.../apache/kylin/dict/AppendTrieDictionary.java | 46 +----
.../org/apache/kylin/dict/CacheDictionary.java | 107 +++++++++++
.../apache/kylin/dict/DateStrDictionary.java | 1 +
.../org/apache/kylin/dict/NumberDictionary.java | 26 +--
.../apache/kylin/dict/TimeStrDictionary.java | 1 +
.../org/apache/kylin/dict/TrieDictionary.java | 137 +-------------
.../apache/kylin/dict/TrieDictionaryForest.java | 45 +----
.../kylin/dict/TrieDictionaryForestBuilder.java | 1 -
.../MultipleDictionaryValueEnumeratorTest.java | 1 +
.../dict/TrieDictionaryForestBenchmark.java | 180 +++++++++++++++++++
.../kylin/dict/TrieDictionaryForestTest.java | 35 ++--
13 files changed, 325 insertions(+), 259 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/1d6a36bf/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 d73b694..bb8880b 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
@@ -217,7 +217,7 @@ abstract public class KylinConfigBase implements Serializable {
// ============================================================================
public boolean isUseForestTrieDictionary() {
- return Boolean.parseBoolean(getOptional("kylin.dictionary.use-forest-trie", "false"));
+ return Boolean.parseBoolean(getOptional("kylin.dictionary.use-forest-trie", "true"));
}
public int getTrieDictionaryForestMaxTrieSizeMB() {
http://git-wip-us.apache.org/repos/asf/kylin/blob/1d6a36bf/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java b/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java
index 1e172bc..03996a7 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/Dictionary.java
@@ -254,4 +254,6 @@ abstract public class Dictionary<T> implements Serializable {
*/
public abstract void readFields(DataInput in) throws IOException;
+
+
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/1d6a36bf/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
index 32bfde6..503c29e 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
@@ -27,10 +27,8 @@ import java.io.DataOutputStream;
import java.io.IOException;
import java.io.PrintStream;
import java.io.UnsupportedEncodingException;
-import java.lang.ref.SoftReference;
import java.util.ArrayList;
import java.util.Arrays;
-import java.util.HashMap;
import java.util.HashSet;
import java.util.IdentityHashMap;
import java.util.LinkedList;
@@ -76,7 +74,7 @@ import org.slf4j.LoggerFactory;
* @author sunyerui
*/
@SuppressWarnings({ "rawtypes", "unchecked", "serial" })
-public class AppendTrieDictionary<T> extends Dictionary<T> {
+public class AppendTrieDictionary<T> extends CacheDictionary<T> {
public static final byte[] HEAD_MAGIC = new byte[] { 0x41, 0x70, 0x70, 0x65, 0x63, 0x64, 0x54, 0x72, 0x69, 0x65, 0x44, 0x69, 0x63, 0x74 }; // "AppendTrieDict"
public static final int HEAD_SIZE_I = HEAD_MAGIC.length;
@@ -87,22 +85,16 @@ public class AppendTrieDictionary<T> extends Dictionary<T> {
private static final Logger logger = LoggerFactory.getLogger(AppendTrieDictionary.class);
transient private String baseDir;
- transient private int baseId;
transient private int maxId;
transient private int maxValueLength;
transient private int nValues;
- transient private BytesConverter<T> bytesConverter;
volatile private TreeMap<DictSliceKey, DictSlice> dictSliceMap;
- transient private boolean enableValueCache = true;
- transient private SoftReference<HashMap> valueToIdCache;
// Constructor both for build and deserialize
public AppendTrieDictionary() {
- if (enableValueCache) {
- valueToIdCache = new SoftReference<>(new HashMap());
- }
+ enableCache();
}
public void initParams(String baseDir, int baseId, int maxId, int maxValueLength, int nValues, BytesConverter bytesConverter) throws IOException {
@@ -111,7 +103,7 @@ public class AppendTrieDictionary<T> extends Dictionary<T> {
this.maxId = maxId;
this.maxValueLength = maxValueLength;
this.nValues = nValues;
- this.bytesConverter = bytesConverter;
+ this.bytesConvert = bytesConverter;
}
public void initDictSliceMap(CachedTreeMap dictMap) throws IOException {
@@ -893,7 +885,7 @@ public class AppendTrieDictionary<T> extends Dictionary<T> {
} else {
logger.info("GlobalDict {} exist, append value", resourcePath);
builder = new Builder<>(resourcePath, dictToUse, dictToUse.baseDir, dictToUse.maxId, dictToUse.maxValueLength,
- dictToUse.nValues, dictToUse.bytesConverter, dictToUse.writeDictMap());
+ dictToUse.nValues, dictToUse.bytesConvert, dictToUse.writeDictMap());
}
return builder;
@@ -1156,31 +1148,6 @@ public class AppendTrieDictionary<T> extends Dictionary<T> {
return maxValueLength;
}
- @Override
- final protected int getIdFromValueImpl(T value, int roundingFlag) {
- if (enableValueCache && roundingFlag == 0) {
- HashMap cache = valueToIdCache.get(); // SoftReference to skip cache gracefully when short of memory
- if (cache != null) {
- Integer id = null;
- id = (Integer) cache.get(value);
- if (id != null)
- return id.intValue();
-
- byte[] valueBytes = bytesConverter.convertToBytes(value);
- id = getIdFromValueBytes(valueBytes, 0, valueBytes.length, roundingFlag);
-
- cache.put(value, id);
- return id;
- }
- }
- byte[] valueBytes = bytesConverter.convertToBytes(value);
- return getIdFromValueBytes(valueBytes, 0, valueBytes.length, roundingFlag);
- }
-
- @Override
- final protected T getValueFromIdImpl(int id) {
- throw new UnsupportedOperationException("AppendTrieDictionary can't retrive value from id");
- }
@Override
protected byte[] getValueBytesFromIdImpl(int id) {
@@ -1198,7 +1165,7 @@ public class AppendTrieDictionary<T> extends Dictionary<T> {
indexOut.writeInt(maxId);
indexOut.writeInt(maxValueLength);
indexOut.writeInt(nValues);
- indexOut.writeUTF(bytesConverter.getClass().getName());
+ indexOut.writeUTF(bytesConvert.getClass().getName());
dictSliceMap.write(indexOut);
dictSliceMap.commit(keepAppend);
}
@@ -1208,7 +1175,7 @@ public class AppendTrieDictionary<T> extends Dictionary<T> {
public AppendTrieDictionary copyToAnotherMeta(KylinConfig srcConfig, KylinConfig dstConfig) throws IOException {
Configuration conf = new Configuration();
AppendTrieDictionary newDict = new AppendTrieDictionary();
- newDict.initParams(baseDir.replaceFirst(srcConfig.getHdfsWorkingDirectory(), dstConfig.getHdfsWorkingDirectory()), baseId, maxId, maxValueLength, nValues, bytesConverter);
+ newDict.initParams(baseDir.replaceFirst(srcConfig.getHdfsWorkingDirectory(), dstConfig.getHdfsWorkingDirectory()), baseId, maxId, maxValueLength, nValues, bytesConvert);
newDict.initDictSliceMap((CachedTreeMap)dictSliceMap);
logger.info("Copy AppendDict from {} to {}", this.baseDir, newDict.baseDir);
Path srcPath = new Path(this.baseDir);
@@ -1256,6 +1223,7 @@ public class AppendTrieDictionary<T> extends Dictionary<T> {
}
}
+
@Override
public void dump(PrintStream out) {
out.println("Total " + nValues + " values, " + (dictSliceMap == null ? 0 : dictSliceMap.size()) + " slice");
http://git-wip-us.apache.org/repos/asf/kylin/blob/1d6a36bf/core-dictionary/src/main/java/org/apache/kylin/dict/CacheDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/CacheDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/CacheDictionary.java
new file mode 100644
index 0000000..575358e
--- /dev/null
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/CacheDictionary.java
@@ -0,0 +1,107 @@
+/*
+ * 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.dict;
+
+import org.apache.kylin.common.util.Dictionary;
+
+import java.lang.ref.SoftReference;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Created by xiefan on 16-12-30.
+ */
+abstract public class CacheDictionary<T> extends Dictionary<T> {
+ private static final long serialVersionUID = 1L;
+
+ transient protected boolean enableValueCache = true;
+
+ transient private SoftReference<Map> valueToIdCache;
+
+ transient private SoftReference<Object[]> idToValueCache;
+
+ transient protected int baseId;
+
+ transient protected BytesConverter<T> bytesConvert;
+
+ public CacheDictionary() {
+
+ }
+
+ //value --> id
+ @Override
+ final protected int getIdFromValueImpl(T value, int roundingFlag) {
+ if (enableValueCache && roundingFlag == 0) {
+ Map cache = valueToIdCache.get(); // SoftReference to skip cache gracefully when short of memory
+ if (cache != null) {
+ Integer id = null;
+ id = (Integer) cache.get(value);
+ if (id != null)
+ return id.intValue();
+ byte[] valueBytes = bytesConvert.convertToBytes(value);
+ id = getIdFromValueBytes(valueBytes, 0, valueBytes.length, roundingFlag);
+ cache.put(value, id);
+ return id;
+ }
+ }
+ byte[] valueBytes = bytesConvert.convertToBytes(value);
+ return getIdFromValueBytes(valueBytes, 0, valueBytes.length, roundingFlag);
+ }
+
+ //id --> value
+ @Override
+ final protected T getValueFromIdImpl(int id) {
+ if (enableValueCache) {
+ Object[] cache = idToValueCache.get();
+ if (cache != null) {
+ int seq = calcSeqNoFromId(id);
+ if (cache[seq] != null)
+ return (T) cache[seq];
+ byte[] valueBytes = getValueBytesFromIdImpl(id);
+ T value = bytesConvert.convertFromBytes(valueBytes, 0, valueBytes.length);
+ cache[seq] = value;
+ return value;
+ }
+ }
+ byte[] valueBytes = getValueBytesFromIdImpl(id);
+ return bytesConvert.convertFromBytes(valueBytes, 0, valueBytes.length);
+ }
+
+ final protected int calcSeqNoFromId(int id) {
+ int seq = id - baseId;
+ if (seq < 0 || seq >= getSize()) {
+ throw new IllegalArgumentException("Not a valid ID: " + id);
+ }
+ return seq;
+ }
+
+ final public void enableCache() {
+ this.enableValueCache = true;
+ if (this.valueToIdCache == null)
+ this.valueToIdCache = new SoftReference<Map>(new ConcurrentHashMap());
+ if (this.idToValueCache == null)
+ this.idToValueCache = new SoftReference<Object[]>(new Object[getSize()]);
+ }
+
+ final public void disableCache() {
+ this.enableValueCache = false;
+ this.valueToIdCache = null;
+ this.idToValueCache = null;
+ }
+}
http://git-wip-us.apache.org/repos/asf/kylin/blob/1d6a36bf/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
index ee8534f..29bbee2 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/DateStrDictionary.java
@@ -156,6 +156,7 @@ public class DateStrDictionary extends Dictionary<String> {
init(pattern, baseId);
}
+
@Override
public int hashCode() {
return 31 * baseId + pattern.hashCode();
http://git-wip-us.apache.org/repos/asf/kylin/blob/1d6a36bf/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java
index 12efbd3..f1b1b3d 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java
@@ -194,30 +194,6 @@ public class NumberDictionary<T> extends TrieDictionary<T> {
return codec.decodeNumber(returnValue, offset);
}
- @Override
- public void enableIdToValueBytesCache() {
- enableIdToValueBytesCache(new EnableIdToValueBytesCacheVisitor() {
- NumberBytesCodec codec = getCodec();
- byte[] tmp = new byte[getSizeOfValue()];
-
- @Override
- public byte[] getBuffer() {
- return codec.buf;
- }
-
- @Override
- public byte[] makeValueBytes(byte[] buf, int length) {
- // the given buf is the codec buf, which we returned in getBuffer()
- codec.bufOffset = 0;
- codec.bufLen = length;
- int numLen = codec.decodeNumber(tmp, 0);
-
- byte[] result = new byte[numLen];
- System.arraycopy(tmp, 0, result, 0, numLen);
- return result;
- }
- });
- }
public static void main(String[] args) throws Exception {
NumberDictionaryBuilder<String> b = new NumberDictionaryBuilder<String>(new StringBytesConverter());
@@ -227,7 +203,7 @@ public class NumberDictionary<T> extends TrieDictionary<T> {
b.addValue("7");
TrieDictionary<String> dict = b.build(0);
- dict.enableIdToValueBytesCache();
+ //dict.enableIdToValueBytesCache();
for (int i = 0; i <= dict.getMaxId(); i++) {
System.out.println(Bytes.toString(dict.getValueBytesFromId(i)));
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/1d6a36bf/core-dictionary/src/main/java/org/apache/kylin/dict/TimeStrDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/TimeStrDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/TimeStrDictionary.java
index fc3db5f..eabc9f1 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/TimeStrDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/TimeStrDictionary.java
@@ -147,4 +147,5 @@ public class TimeStrDictionary extends Dictionary<String> {
@Override
public void readFields(DataInput in) throws IOException {
}
+
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/1d6a36bf/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
index c099de0..957207e 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionary.java
@@ -27,10 +27,8 @@ import java.io.IOException;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.io.PrintStream;
-import java.lang.ref.SoftReference;
import java.util.Arrays;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
+
import org.apache.kylin.common.util.Bytes;
import org.apache.kylin.common.util.BytesUtil;
@@ -55,7 +53,7 @@ import com.google.common.base.Preconditions;
* @author yangli9
*/
@SuppressWarnings({ "rawtypes", "unchecked" })
-public class TrieDictionary<T> extends Dictionary<T> {
+public class TrieDictionary<T> extends CacheDictionary<T> {
private static final long serialVersionUID = 1L;
public static final byte[] MAGIC = new byte[] { 0x54, 0x72, 0x69, 0x65, 0x44, 0x69, 0x63, 0x74 }; // "TrieDict"
@@ -74,21 +72,13 @@ public class TrieDictionary<T> extends Dictionary<T> {
transient private int bodyLen;
transient private int sizeChildOffset;
transient private int sizeNoValuesBeneath;
- transient private int baseId;
transient private int maxValueLength;
- transient private BytesConverter<T> bytesConvert;
transient private int nValues;
transient private int sizeOfId;
transient private long childOffsetMask;
transient private int firstByteOffset;
- transient private boolean enableValueCache = true;
- transient private SoftReference<Map> valueToIdCache;
- transient private SoftReference<Object[]> idToValueCache;
-
- transient private boolean enableIdToValueBytesCache = false;
- transient private byte[][] idToValueBytesCache;
public TrieDictionary() { // default constructor for Writable interface
}
@@ -120,16 +110,13 @@ public class TrieDictionary<T> extends Dictionary<T> {
this.sizeOfId = BytesUtil.sizeForValue(baseId + nValues + 1); // note baseId could raise 1 byte in ID space, +1 to reserve all 0xFF for NULL case
this.childOffsetMask = ~((long) (BIT_IS_LAST_CHILD | BIT_IS_END_OF_VALUE) << ((sizeChildOffset - 1) * 8));
this.firstByteOffset = sizeChildOffset + sizeNoValuesBeneath + 1; // the offset from begin of node to its first value byte
+ enableCache();
} catch (Exception e) {
if (e instanceof RuntimeException)
throw (RuntimeException) e;
else
throw new RuntimeException(e);
}
- if (enableValueCache) {
- valueToIdCache = new SoftReference<Map>(new ConcurrentHashMap());
- idToValueCache = new SoftReference<Object[]>(new Object[nValues]);
- }
}
@Override
@@ -152,26 +139,6 @@ public class TrieDictionary<T> extends Dictionary<T> {
return maxValueLength;
}
- @Override
- final protected int getIdFromValueImpl(T value, int roundingFlag) {
- if (enableValueCache && roundingFlag == 0) {
- Map cache = valueToIdCache.get(); // SoftReference to skip cache gracefully when short of memory
- if (cache != null) {
- Integer id = null;
- id = (Integer) cache.get(value);
- if (id != null)
- return id.intValue();
-
- byte[] valueBytes = bytesConvert.convertToBytes(value);
- id = getIdFromValueBytes(valueBytes, 0, valueBytes.length, roundingFlag);
-
- cache.put(value, id);
- return id;
- }
- }
- byte[] valueBytes = bytesConvert.convertToBytes(value);
- return getIdFromValueBytes(valueBytes, 0, valueBytes.length, roundingFlag);
- }
@Override
protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag) {
@@ -267,35 +234,9 @@ public class TrieDictionary<T> extends Dictionary<T> {
return k;
}
- @Override
- final protected T getValueFromIdImpl(int id) {
- if (enableValueCache) {
- Object[] cache = idToValueCache.get(); // SoftReference to skip cache gracefully when short of memory
- if (cache != null) {
- int seq = calcSeqNoFromId(id);
- if (cache[seq] != null)
- return (T) cache[seq];
-
- byte[] value = new byte[getSizeOfValue()];
- int length = getValueBytesFromId(id, value, 0);
- T result = bytesConvert.convertFromBytes(value, 0, length);
-
- cache[seq] = result;
- return result;
- }
- }
- byte[] value = new byte[getSizeOfValue()];
- int length = getValueBytesFromId(id, value, 0);
- return bytesConvert.convertFromBytes(value, 0, length);
- }
@Override
protected byte[] getValueBytesFromIdImpl(int id) {
- if (enableIdToValueBytesCache) {
- int seq = calcSeqNoFromId(id);
- return idToValueBytesCache[seq];
- }
-
byte[] buf = new byte[maxValueLength];
int len = getValueBytesFromIdImpl(id, buf, 0);
@@ -363,68 +304,6 @@ public class TrieDictionary<T> extends Dictionary<T> {
}
}
- public void enableIdToValueBytesCache() {
- enableIdToValueBytesCache(new EnableIdToValueBytesCacheVisitor() {
- @Override
- public byte[] getBuffer() {
- return new byte[getSizeOfValue()];
- }
-
- @Override
- public byte[] makeValueBytes(byte[] buf, int length) {
- byte[] valueBytes = new byte[length];
- System.arraycopy(buf, 0, valueBytes, 0, length);
- return valueBytes;
- }
- });
- }
-
- interface EnableIdToValueBytesCacheVisitor {
- byte[] getBuffer();
-
- byte[] makeValueBytes(byte[] buf, int length);
- }
-
- protected void enableIdToValueBytesCache(EnableIdToValueBytesCacheVisitor visitor) {
- enableIdToValueBytesCache = true;
- idToValueBytesCache = new byte[nValues][];
- enableIdToValueBytesCache_recursion(headSize, 0, visitor.getBuffer(), 0, visitor);
- }
-
- private void enableIdToValueBytesCache_recursion(int n, int seq, byte[] buf, int tail, EnableIdToValueBytesCacheVisitor visitor) {
- // write current node value
- int p = n + firstByteOffset;
- int len = BytesUtil.readUnsigned(trieBytes, p - 1, 1);
- System.arraycopy(trieBytes, p, buf, tail, len);
- tail += len;
-
- // if the value is ended
- boolean isEndOfValue = checkFlag(n, BIT_IS_END_OF_VALUE);
- if (isEndOfValue) {
- idToValueBytesCache[seq] = visitor.makeValueBytes(buf, tail);
- seq++;
- }
-
- // find a child to continue
- int c = getChildOffset(n);
- if (c == headSize) // has no children
- return;
-
- // process each child
- while (true) {
- enableIdToValueBytesCache_recursion(c, seq, buf, tail, visitor);
-
- int nValuesBeneath = BytesUtil.readUnsigned(trieBytes, c + sizeChildOffset, sizeNoValuesBeneath);
- seq += nValuesBeneath;
-
- // go next child
- if (checkFlag(c, BIT_IS_LAST_CHILD))
- break; // no more child? we are done
- p = c + firstByteOffset;
- c = p + BytesUtil.readUnsigned(trieBytes, p - 1, 1);
- }
- }
-
private boolean checkFlag(int offset, int bit) {
return (trieBytes[offset] & bit) > 0;
}
@@ -436,14 +315,6 @@ public class TrieDictionary<T> extends Dictionary<T> {
return baseId + seq;
}
- private int calcSeqNoFromId(int id) {
- int seq = id - baseId;
- if (seq < 0 || seq >= nValues) {
- throw new IllegalArgumentException("Not a valid ID: " + id);
- }
- return seq;
- }
-
@Override
public void write(DataOutput out) throws IOException {
out.write(trieBytes);
@@ -552,7 +423,7 @@ public class TrieDictionary<T> extends Dictionary<T> {
Preconditions.checkArgument(dict2.contains(dict));
Preconditions.checkArgument(dict.equals(dict2));
- dict2.enableIdToValueBytesCache();
+ //dict2.enableIdToValueBytesCache();
for (int i = 0; i <= dict.getMaxId(); i++) {
System.out.println(Bytes.toString(dict.getValueBytesFromId(i)));
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/1d6a36bf/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java
index e746348..c655854 100755
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java
@@ -27,6 +27,7 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
+
import org.apache.kylin.common.util.ByteArray;
import org.apache.kylin.common.util.Bytes;
import org.apache.kylin.common.util.BytesUtil;
@@ -40,18 +41,14 @@ import org.apache.kylin.common.util.Dictionary;
* <p>
* Created by xiefan on 16-10-26.
*/
-public class TrieDictionaryForest<T> extends Dictionary<T> {
+public class TrieDictionaryForest<T> extends CacheDictionary<T> {
private static final long serialVersionUID = 1L;
private ArrayList<TrieDictionary<T>> trees;
private ArrayList<ByteArray> valueDivide;
- private ArrayList<Integer> accuOffset; //find tree
-
- private BytesConverter<T> bytesConvert;
-
- private int baseId;
+ private ArrayList<Integer> accuOffset;
private ArrayList<ByteArray> maxValue;
@@ -66,6 +63,7 @@ public class TrieDictionaryForest<T> extends Dictionary<T> {
this.bytesConvert = bytesConverter;
this.baseId = baseId;
initMaxValue();
+ enableCache();
}
@Override
@@ -102,12 +100,6 @@ public class TrieDictionaryForest<T> extends Dictionary<T> {
return maxValue;
}
- // value --> id
- @Override
- protected int getIdFromValueImpl(T value, int roundingFlag) throws IllegalArgumentException {
- byte[] valueBytes = bytesConvert.convertToBytes(value);
- return getIdFromValueBytesImpl(valueBytes, 0, valueBytes.length, roundingFlag);
- }
@Override
protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag) throws IllegalArgumentException {
@@ -148,15 +140,7 @@ public class TrieDictionaryForest<T> extends Dictionary<T> {
return id;
}
- @Override
- protected T getValueFromIdImpl(int id) throws IllegalArgumentException {
- byte[] data = getValueBytesFromIdImpl(id);
- if (data != null) {
- return bytesConvert.convertFromBytes(data, 0, data.length);
- } else {
- return null;
- }
- }
+
@Override
protected int getValueBytesFromIdImpl(int id, byte[] returnValue, int offset) throws IllegalArgumentException {
@@ -271,6 +255,7 @@ public class TrieDictionaryForest<T> extends Dictionary<T> {
trees.add(dict);
}
initMaxValue();
+ enableCache();
} catch (Exception e) {
if (e instanceof RuntimeException)
throw (RuntimeException) e;
@@ -383,22 +368,4 @@ public class TrieDictionaryForest<T> extends Dictionary<T> {
}
}
- public static void main(String[] args) {
- ArrayList<String> list = new ArrayList<>();
- list.add("\u4e00");
- list.add("\u4e8c");
- list.add("\u4e09");
- list.add("");
- list.add("part");
- list.add("par");
- list.add("partition");
- list.add("party");
- list.add("parties");
- list.add("paint");
- Collections.sort(list);
- for (String str : list) {
- System.out.println("found value:" + str + " index:" + lowerBound(str, list));
- }
- }
-
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/1d6a36bf/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForestBuilder.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForestBuilder.java b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForestBuilder.java
index 4ee30f0..af2e302 100755
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForestBuilder.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForestBuilder.java
@@ -111,7 +111,6 @@ public class TrieDictionaryForestBuilder<T> {
reset();
}
TrieDictionaryForest<T> forest = new TrieDictionaryForest<T>(this.trees, this.valueDivide, this.accuOffset, this.bytesConverter, baseId);
-
// if input values are not in ascending order and tree num>1,TrieDictionaryForest can not work correctly.
if (forest.getTrees().size() > 1 && !isOrdered) {
throw new IllegalStateException("Invalid input data. Unordered data can not be split into multi trees");
http://git-wip-us.apache.org/repos/asf/kylin/blob/1d6a36bf/core-dictionary/src/test/java/org/apache/kylin/dict/MultipleDictionaryValueEnumeratorTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/MultipleDictionaryValueEnumeratorTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/MultipleDictionaryValueEnumeratorTest.java
index ad166c2..73e0935 100644
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/MultipleDictionaryValueEnumeratorTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/MultipleDictionaryValueEnumeratorTest.java
@@ -158,6 +158,7 @@ public class MultipleDictionaryValueEnumeratorTest {
@Override
public void readFields(DataInput in) throws IOException {}
+
@Override
public boolean contains(Dictionary another) {
return false;
http://git-wip-us.apache.org/repos/asf/kylin/blob/1d6a36bf/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestBenchmark.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestBenchmark.java b/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestBenchmark.java
new file mode 100644
index 0000000..0b4c0e3
--- /dev/null
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestBenchmark.java
@@ -0,0 +1,180 @@
+/*
+ * 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.dict;
+
+import org.apache.kylin.common.util.Dictionary;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Random;
+import java.util.UUID;
+
+/**
+ * Created by xiefan on 16-12-28.
+ */
+@Ignore
+public class TrieDictionaryForestBenchmark {
+
+ private static final Random rand = new Random(System.currentTimeMillis());
+
+ private CacheDictionary<String> oldDict;
+
+ private CacheDictionary<String> newDict;
+
+ private ArrayList<String> rawData;
+
+ private int cardnality = 100;
+
+ private int testTimes = 100000;
+
+ @Before
+ public void before() {
+ int dataSize = 100 * 10000;
+ TrieDictionaryBuilder<String> b1 = new TrieDictionaryBuilder<>(new StringBytesConverter());
+ TrieDictionaryForestBuilder<String> b2 = new TrieDictionaryForestBuilder<String>(new StringBytesConverter(), 0, 5);
+ this.rawData = genStringDataSet(dataSize);
+ for (String str : this.rawData) {
+ b1.addValue(str);
+ b2.addValue(str);
+ }
+ this.oldDict = b1.build(0);
+ this.newDict = b2.build();
+ System.out.println("new dict split tree size : " + ((TrieDictionaryForest<String>) newDict).getTrees().size());
+ }
+
+ @Test
+ public void testAll() {
+ benchmarkWithoutCache();
+ benchmarkWithCache();
+ }
+
+ @Test
+ public void benchmarkWithoutCache() {
+ oldDict.disableCache();
+ newDict.disableCache();
+ runBenchmark("benchmarkWithoutCache");
+ }
+
+ @Test
+ public void benchmarkWithCache() {
+ oldDict.enableCache();
+ newDict.enableCache();
+ runBenchmark("benchmarkWithCache");
+ }
+
+ private void runBenchmark(String testName) {
+ long oldTime = runQueryValue(oldDict, cardnality, testTimes);
+ long oldTime2 = runQueryId(rawData, oldDict, cardnality, testTimes);
+ long oldTime3 = runQueryValueBytes(oldDict, cardnality, testTimes);
+ long oldTime4 = runQueryValueBytes2(oldDict, cardnality, testTimes);
+ long oldTime5 = runQueryIdByValueBytes(rawData, oldDict, cardnality, testTimes);
+ long newTime = runQueryValue(newDict, cardnality, testTimes);
+ long newTime2 = runQueryId(rawData, newDict, cardnality, testTimes);
+ long newTime3 = runQueryValueBytes(newDict, cardnality, testTimes);
+ long newTime4 = runQueryValueBytes2(newDict, cardnality, testTimes);
+ long newTime5 = runQueryIdByValueBytes(rawData, newDict, cardnality, testTimes);
+ System.out.println(testName);
+ System.out.println("old dict value --> id : " + oldTime2);
+ System.out.println("new dict value --> id :" + newTime2);
+ System.out.println("old dict value bytes --> id : " + oldTime5);
+ System.out.println("new dict value bytes--> id :" + newTime5);
+ System.out.println("old dict id --> value : " + oldTime);
+ System.out.println("new dict id --> value : " + newTime);
+ System.out.println("old dict id --> value bytes : " + oldTime3);
+ System.out.println("new dict id --> value bytes : " + newTime3);
+ System.out.println("old dict id --> value bytes (method 2): " + oldTime4);
+ System.out.println("new dict id --> value bytes (method 2): " + newTime4);
+ }
+
+ //id -- value
+ private long runQueryValue(Dictionary<String> dict, int cardnality, int testTimes) {
+ long startTime = System.currentTimeMillis();
+ int step = 1;
+ for (int i = 0; i < testTimes; i++) {
+ for (int j = 0; j < cardnality; j++) {
+ step |= dict.getValueFromId(j).length();
+ }
+ }
+ return System.currentTimeMillis() - startTime;
+ }
+
+ private long runQueryValueBytes(Dictionary<String> dict, int cardnality, int testTimes) {
+ long startTime = System.currentTimeMillis();
+ int step = 1;
+ for (int i = 0; i < testTimes; i++) {
+ for (int j = 0; j < cardnality; j++) {
+ step |= dict.getValueBytesFromId(j).length;
+ }
+ }
+ return System.currentTimeMillis() - startTime;
+ }
+
+ private long runQueryValueBytes2(Dictionary<String> dict, int cardnality, int testTimes) {
+ long startTime = System.currentTimeMillis();
+ int step = 1;
+ byte[] returnValue = new byte[2048];
+ for (int i = 0; i < testTimes; i++) {
+ for (int j = 0; j < cardnality; j++) {
+ int size = dict.getValueBytesFromId(j, returnValue, 0);
+ step |= size;
+ }
+ }
+ return System.currentTimeMillis() - startTime;
+ }
+
+ private long runQueryId(ArrayList<String> rawData, Dictionary<String> dict, int cardnality, int testTimes) {
+ long startTime = System.currentTimeMillis();
+ int step = 1;
+ for (int i = 0; i < testTimes; i++) {
+ for (int j = 0; j < cardnality; j++) {
+ step |= dict.getIdFromValue(rawData.get(j));
+ }
+ }
+ return System.currentTimeMillis() - startTime;
+ }
+
+ private long runQueryIdByValueBytes(ArrayList<String> rawData, Dictionary<String> dict, int cardnality, int testTimes) {
+ List<byte[]> testBytes = new ArrayList<>();
+ StringBytesConverter converter = new StringBytesConverter();
+ for (int i = 0; i < cardnality; i++) {
+ testBytes.add(converter.convertToBytes(rawData.get(i)));
+ }
+ long startTime = System.currentTimeMillis();
+ int step = 1;
+ for (int i = 0; i < testTimes; i++) {
+ for (int j = 0; j < cardnality; j++) {
+ step |= dict.getIdFromValueBytes(testBytes.get(j), 0, testBytes.get(j).length);
+ }
+ }
+ return System.currentTimeMillis() - startTime;
+ }
+
+ private ArrayList<String> genStringDataSet(int totalSize) {
+ ArrayList<String> data = new ArrayList<>();
+ for (int i = 0; i < totalSize; i++) {
+ data.add(UUID.randomUUID().toString());
+ }
+ Collections.sort(data);
+ return data;
+ }
+}
http://git-wip-us.apache.org/repos/asf/kylin/blob/1d6a36bf/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestTest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestTest.java b/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestTest.java
index ee092c9..68cf301 100755
--- a/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestTest.java
+++ b/core-dictionary/src/test/java/org/apache/kylin/dict/TrieDictionaryForestTest.java
@@ -58,7 +58,7 @@ public class TrieDictionaryForestTest {
TrieDictionaryForest<String> dict = builder.build();
assertSameBehaviorAsTrie(dict, strs, 0);
}
-
+
@Test
public void testBasicFound() {
ArrayList<String> strs = new ArrayList<String>();
@@ -106,7 +106,7 @@ public class TrieDictionaryForestTest {
assertEquals(expectId, dict.getIdFromValue(s));
expectId++;
}
-
+
assertSameBehaviorAsTrie(dict, strs, baseId);
}
@@ -128,7 +128,7 @@ public class TrieDictionaryForestTest {
assertEquals(255, id);
id = dict.getIdFromValue(null, -1);
assertEquals(255, id);
-
+
assertSameBehaviorAsTrie(dict, strs, 0);
}
@@ -263,7 +263,6 @@ public class TrieDictionaryForestTest {
}
@Test
- @Ignore
public void categoryNamesTest() throws Exception {
InputStream is = new FileInputStream("src/test/resources/dict/dw_category_grouping_names.dat");
ArrayList<String> str = loadStrings(is);
@@ -284,19 +283,19 @@ public class TrieDictionaryForestTest {
}
@Test
- public void emptyDictTest() throws Exception{
+ public void emptyDictTest() throws Exception {
TrieDictionaryForestBuilder<String> b = new TrieDictionaryForestBuilder<String>(new StringBytesConverter());
TrieDictionaryForest<String> dict = b.build();
- try{
+ try {
int id = dict.getIdFromValue("123", 0);
fail("id should not exist");
- }catch (IllegalArgumentException e){
+ } catch (IllegalArgumentException e) {
//right
}
- try{
+ try {
String value = dict.getValueFromIdImpl(123);
fail("value should not exist");
- }catch (IllegalArgumentException e){
+ } catch (IllegalArgumentException e) {
//right
}
}
@@ -732,12 +731,6 @@ public class TrieDictionaryForestTest {
System.out.println("compare build time. Old trie : " + oldDictTotalBuildTime / 1000.0 + "s.New trie : " + newDictTotalBuildTime / 1000.0 + "s");
}
- @Test
- public void queryTimeBenchmarkTest() throws Exception {
- int count = (int) (Integer.MAX_VALUE * 0.8 / 640);
- benchmarkStringDictionary(new RandomStrings(count));
- }
-
private void evaluateDataSize(ArrayList<String> list) {
long size = 0;
for (String str : list)
@@ -867,7 +860,6 @@ public class TrieDictionaryForestTest {
int baseId = new Random().nextInt(100);
TrieDictionaryForestBuilder<String> b = newDictBuilder(str, baseId, 2);
TrieDictionaryForest<String> dict = b.build();
- //dict.dump(System.out);
TreeSet<String> set = new TreeSet<String>();
for (String s : str) {
set.add(s);
@@ -881,7 +873,7 @@ public class TrieDictionaryForestTest {
int id = baseId;
for (; it.hasNext(); id++) {
String value = it.next();
- // System.out.println("checking " + id + " <==> " + value);
+ //System.out.println("checking " + id + " <==> " + value);
assertEquals(id, dict.getIdFromValue(value));
assertEquals(value, dict.getValueFromId(id));
@@ -892,7 +884,7 @@ public class TrieDictionaryForestTest {
for (String s : notFound) {
try {
int nullId = dict.getIdFromValue(s);
- System.out.println("null value id:" + nullId);
+ //System.out.println("null value id:" + nullId);
fail("For not found value '" + s + "', IllegalArgumentException is expected");
} catch (IllegalArgumentException e) {
// good
@@ -938,8 +930,9 @@ public class TrieDictionaryForestTest {
public static TrieDictionaryForestBuilder<String> newDictBuilder(Iterable<String> strs, int baseId, int treeSize) {
TrieDictionaryForestBuilder<String> b = new TrieDictionaryForestBuilder<String>(new StringBytesConverter(), baseId);
b.setMaxTrieTreeSize(treeSize);
- for (String s : strs)
+ for (String s : strs) {
b.addValue(s);
+ }
return b;
}
@@ -950,7 +943,7 @@ public class TrieDictionaryForestTest {
b.addValue(strs.next());
return b;
}
-
+
private static class RandomStrings implements Iterable<String> {
final private int size;
@@ -1039,7 +1032,7 @@ public class TrieDictionaryForestTest {
trieBuilder.addValue(s);
}
TrieDictionary<String> trie = trieBuilder.build(baseId);
-
+
assertEquals(trie.getMaxId(), dict.getMaxId());
assertEquals(trie.getMinId(), dict.getMinId());
assertEquals(trie.getSize(), dict.getSize());
[08/17] kylin git commit: KYLIN-2282 Step name Build N-Dimension
Cuboid Data : N-Dimension is inaccurate
Posted by sh...@apache.org.
KYLIN-2282 Step name Build N-Dimension Cuboid Data : N-Dimension is inaccurate
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/809b7830
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/809b7830
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/809b7830
Branch: refs/heads/sparkcubing-rebase
Commit: 809b78309d0b85058aad43e6b863271dfebd1255
Parents: e043d5f
Author: shaofengshi <sh...@apache.org>
Authored: Thu Jan 5 12:00:12 2017 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Jan 5 12:01:36 2017 +0800
----------------------------------------------------------------------
.../kylin/engine/mr/BatchCubingJobBuilder.java | 27 +++++++++----------
.../kylin/engine/mr/BatchCubingJobBuilder2.java | 28 +++++++++-----------
.../kylin/engine/mr/JobBuilderSupport.java | 11 ++++----
3 files changed, 30 insertions(+), 36 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/809b7830/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
index 6c973eb..8fbecd3 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
@@ -63,16 +63,15 @@ public class BatchCubingJobBuilder extends JobBuilderSupport {
result.addTask(createBuildDictionaryStep(jobId));
// Phase 3: Build Cube
- RowKeyDesc rowKeyDesc = ((CubeSegment) seg).getCubeDesc().getRowkey();
- final int groupRowkeyColumnsCount = ((CubeSegment) seg).getCubeDesc().getBuildLevel();
+ RowKeyDesc rowKeyDesc = seg.getCubeDesc().getRowkey();
+ final int groupRowkeyColumnsCount = seg.getCubeDesc().getBuildLevel();
final int totalRowkeyColumnsCount = rowKeyDesc.getRowKeyColumns().length;
- final String[] cuboidOutputTempPath = getCuboidOutputPaths(cuboidRootPath, totalRowkeyColumnsCount, groupRowkeyColumnsCount);
+ final String[] cuboidOutputTempPath = getCuboidOutputPaths(cuboidRootPath, groupRowkeyColumnsCount);
// base cuboid step
- result.addTask(createBaseCuboidStep(cuboidOutputTempPath, jobId));
+ result.addTask(createBaseCuboidStep(cuboidOutputTempPath[0], jobId));
// n dim cuboid steps
- for (int i = 1; i <= groupRowkeyColumnsCount; i++) {
- int dimNum = totalRowkeyColumnsCount - i;
- result.addTask(createNDimensionCuboidStep(cuboidOutputTempPath, dimNum, totalRowkeyColumnsCount));
+ for (int i = 1; i < groupRowkeyColumnsCount; i++) {
+ result.addTask(createNDimensionCuboidStep(cuboidOutputTempPath[i - 1], cuboidOutputTempPath[i], i));
}
outputSide.addStepPhase3_BuildCube(result, cuboidRootPath);
@@ -84,7 +83,7 @@ public class BatchCubingJobBuilder extends JobBuilderSupport {
return result;
}
- private MapReduceExecutable createBaseCuboidStep(String[] cuboidOutputTempPath, String jobId) {
+ private MapReduceExecutable createBaseCuboidStep(String cuboidOutputPath, String jobId) {
// base cuboid job
MapReduceExecutable baseCuboidStep = new MapReduceExecutable();
@@ -96,7 +95,7 @@ public class BatchCubingJobBuilder extends JobBuilderSupport {
appendExecCmdParameters(cmd, BatchConstants.ARG_CUBE_NAME, seg.getRealization().getName());
appendExecCmdParameters(cmd, BatchConstants.ARG_SEGMENT_ID, seg.getUuid());
appendExecCmdParameters(cmd, BatchConstants.ARG_INPUT, "FLAT_TABLE"); // marks flat table input
- appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, cuboidOutputTempPath[0]);
+ appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, cuboidOutputPath);
appendExecCmdParameters(cmd, BatchConstants.ARG_JOB_NAME, "Kylin_Base_Cuboid_Builder_" + seg.getRealization().getName());
appendExecCmdParameters(cmd, BatchConstants.ARG_LEVEL, "0");
appendExecCmdParameters(cmd, BatchConstants.ARG_CUBING_JOB_ID, jobId);
@@ -107,20 +106,20 @@ public class BatchCubingJobBuilder extends JobBuilderSupport {
return baseCuboidStep;
}
- private MapReduceExecutable createNDimensionCuboidStep(String[] cuboidOutputTempPath, int dimNum, int totalRowkeyColumnCount) {
+ private MapReduceExecutable createNDimensionCuboidStep(String parentPath, String outputPath, int level) {
// ND cuboid job
MapReduceExecutable ndCuboidStep = new MapReduceExecutable();
- ndCuboidStep.setName(ExecutableConstants.STEP_NAME_BUILD_N_D_CUBOID + " : " + dimNum + "-Dimension");
+ ndCuboidStep.setName(ExecutableConstants.STEP_NAME_BUILD_N_D_CUBOID + " : level " + level);
StringBuilder cmd = new StringBuilder();
appendMapReduceParameters(cmd);
appendExecCmdParameters(cmd, BatchConstants.ARG_CUBE_NAME, seg.getRealization().getName());
appendExecCmdParameters(cmd, BatchConstants.ARG_SEGMENT_ID, seg.getUuid());
- appendExecCmdParameters(cmd, BatchConstants.ARG_INPUT, cuboidOutputTempPath[totalRowkeyColumnCount - dimNum - 1]);
- appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, cuboidOutputTempPath[totalRowkeyColumnCount - dimNum]);
+ appendExecCmdParameters(cmd, BatchConstants.ARG_INPUT, parentPath);
+ appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, outputPath);
appendExecCmdParameters(cmd, BatchConstants.ARG_JOB_NAME, "Kylin_ND-Cuboid_Builder_" + seg.getRealization().getName() + "_Step");
- appendExecCmdParameters(cmd, BatchConstants.ARG_LEVEL, "" + (totalRowkeyColumnCount - dimNum));
+ appendExecCmdParameters(cmd, BatchConstants.ARG_LEVEL, "" + level);
ndCuboidStep.setMapReduceParams(cmd.toString());
ndCuboidStep.setMapReduceJobClass(NDCuboidJob.class);
http://git-wip-us.apache.org/repos/asf/kylin/blob/809b7830/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
index 966bb1b..20540a6 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
@@ -19,7 +19,6 @@
package org.apache.kylin.engine.mr;
import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.model.RowKeyDesc;
import org.apache.kylin.engine.mr.IMRInput.IMRBatchCubingInputSide;
import org.apache.kylin.engine.mr.IMROutput2.IMRBatchCubingOutputSide2;
import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
@@ -77,16 +76,13 @@ public class BatchCubingJobBuilder2 extends JobBuilderSupport {
}
private void addLayerCubingSteps(final CubingJob result, final String jobId, final String cuboidRootPath) {
- RowKeyDesc rowKeyDesc = ((CubeSegment) seg).getCubeDesc().getRowkey();
- final int groupRowkeyColumnsCount = ((CubeSegment) seg).getCubeDesc().getBuildLevel();
- final int totalRowkeyColumnsCount = rowKeyDesc.getRowKeyColumns().length;
- final String[] cuboidOutputTempPath = getCuboidOutputPaths(cuboidRootPath, totalRowkeyColumnsCount, groupRowkeyColumnsCount);
+ final int maxLevel = seg.getCubeDesc().getBuildLevel();
+ final String[] cuboidOutputTempPath = getCuboidOutputPaths(cuboidRootPath, maxLevel);
// base cuboid step
- result.addTask(createBaseCuboidStep(cuboidOutputTempPath, jobId));
+ result.addTask(createBaseCuboidStep(cuboidOutputTempPath[0], jobId));
// n dim cuboid steps
- for (int i = 1; i < groupRowkeyColumnsCount; i++) {
- int dimNum = totalRowkeyColumnsCount - i;
- result.addTask(createNDimensionCuboidStep(cuboidOutputTempPath, dimNum, totalRowkeyColumnsCount, jobId));
+ for (int i = 1; i < maxLevel; i++) {
+ result.addTask(createNDimensionCuboidStep(cuboidOutputTempPath[i - 1], cuboidOutputTempPath[i], i, jobId));
}
}
@@ -125,7 +121,7 @@ public class BatchCubingJobBuilder2 extends JobBuilderSupport {
return InMemCuboidJob.class;
}
- private MapReduceExecutable createBaseCuboidStep(String[] cuboidOutputTempPath, String jobId) {
+ private MapReduceExecutable createBaseCuboidStep(String cuboidOutputPath, String jobId) {
// base cuboid job
MapReduceExecutable baseCuboidStep = new MapReduceExecutable();
@@ -137,7 +133,7 @@ public class BatchCubingJobBuilder2 extends JobBuilderSupport {
appendExecCmdParameters(cmd, BatchConstants.ARG_CUBE_NAME, seg.getRealization().getName());
appendExecCmdParameters(cmd, BatchConstants.ARG_SEGMENT_ID, seg.getUuid());
appendExecCmdParameters(cmd, BatchConstants.ARG_INPUT, "FLAT_TABLE"); // marks flat table input
- appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, cuboidOutputTempPath[0]);
+ appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, cuboidOutputPath);
appendExecCmdParameters(cmd, BatchConstants.ARG_JOB_NAME, "Kylin_Base_Cuboid_Builder_" + seg.getRealization().getName());
appendExecCmdParameters(cmd, BatchConstants.ARG_LEVEL, "0");
appendExecCmdParameters(cmd, BatchConstants.ARG_CUBING_JOB_ID, jobId);
@@ -152,20 +148,20 @@ public class BatchCubingJobBuilder2 extends JobBuilderSupport {
return BaseCuboidJob.class;
}
- private MapReduceExecutable createNDimensionCuboidStep(String[] cuboidOutputTempPath, int dimNum, int totalRowkeyColumnCount, String jobId) {
+ private MapReduceExecutable createNDimensionCuboidStep(String parentPath, String outputPath, int level, String jobId) {
// ND cuboid job
MapReduceExecutable ndCuboidStep = new MapReduceExecutable();
- ndCuboidStep.setName(ExecutableConstants.STEP_NAME_BUILD_N_D_CUBOID + " : " + dimNum + "-Dimension");
+ ndCuboidStep.setName(ExecutableConstants.STEP_NAME_BUILD_N_D_CUBOID + " : level " + level);
StringBuilder cmd = new StringBuilder();
appendMapReduceParameters(cmd);
appendExecCmdParameters(cmd, BatchConstants.ARG_CUBE_NAME, seg.getRealization().getName());
appendExecCmdParameters(cmd, BatchConstants.ARG_SEGMENT_ID, seg.getUuid());
- appendExecCmdParameters(cmd, BatchConstants.ARG_INPUT, cuboidOutputTempPath[totalRowkeyColumnCount - dimNum - 1]);
- appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, cuboidOutputTempPath[totalRowkeyColumnCount - dimNum]);
+ appendExecCmdParameters(cmd, BatchConstants.ARG_INPUT, parentPath);
+ appendExecCmdParameters(cmd, BatchConstants.ARG_OUTPUT, outputPath);
appendExecCmdParameters(cmd, BatchConstants.ARG_JOB_NAME, "Kylin_ND-Cuboid_Builder_" + seg.getRealization().getName() + "_Step");
- appendExecCmdParameters(cmd, BatchConstants.ARG_LEVEL, "" + (totalRowkeyColumnCount - dimNum));
+ appendExecCmdParameters(cmd, BatchConstants.ARG_LEVEL, "" + level);
appendExecCmdParameters(cmd, BatchConstants.ARG_CUBING_JOB_ID, jobId);
ndCuboidStep.setMapReduceParams(cmd.toString());
http://git-wip-us.apache.org/repos/asf/kylin/blob/809b7830/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
index 5f5814b..23fce2b 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
@@ -192,14 +192,13 @@ public class JobBuilderSupport {
return buf.append(" -").append(paraName).append(" ").append(paraValue);
}
- public String[] getCuboidOutputPaths(String cuboidRootPath, int totalRowkeyColumnCount, int groupRowkeyColumnsCount) {
- String[] paths = new String[groupRowkeyColumnsCount + 1];
- for (int i = 0; i <= groupRowkeyColumnsCount; i++) {
- int dimNum = totalRowkeyColumnCount - i;
- if (dimNum == totalRowkeyColumnCount) {
+ public String[] getCuboidOutputPaths(String cuboidRootPath, int levels) {
+ String[] paths = new String[levels];
+ for (int i = 0; i < levels; i++) {
+ if (i == 0) {
paths[i] = cuboidRootPath + "base_cuboid";
} else {
- paths[i] = cuboidRootPath + dimNum + "d_cuboid";
+ paths[i] = cuboidRootPath + "level_" + i + "_cuboid";
}
}
return paths;
[16/17] kylin git commit: KYLIN-2344 Package spark into Kylin binary
package
Posted by sh...@apache.org.
KYLIN-2344 Package spark into Kylin binary package
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/61015651
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/61015651
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/61015651
Branch: refs/heads/sparkcubing-rebase
Commit: 61015651ab8ff47d16a9c475b02adc7cd7dbe1f9
Parents: a69a3b6
Author: shaofengshi <sh...@apache.org>
Authored: Tue Jan 3 10:28:38 2017 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Jan 5 15:20:48 2017 +0800
----------------------------------------------------------------------
build/bin/check-env.sh | 6 ++
build/bin/find-spark-dependency.sh | 45 +++++++++++++++
build/bin/kylin.sh | 6 ++
build/conf/kylin-spark-conf.properties | 2 +-
build/conf/kylin.properties | 3 -
build/script/compress.sh | 5 +-
build/script/download-spark.sh | 52 +++++++++++++++++
build/script/functions.sh | 60 ++++++++++++++++++++
build/script/package.sh | 1 +
.../org/apache/kylin/common/KylinConfig.java | 14 +++++
.../apache/kylin/common/KylinConfigBase.java | 27 ++++-----
.../spark/SparkBatchCubingJobBuilder2.java | 4 +-
12 files changed, 200 insertions(+), 25 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/61015651/build/bin/check-env.sh
----------------------------------------------------------------------
diff --git a/build/bin/check-env.sh b/build/bin/check-env.sh
index 9cd8a64..8904557 100644
--- a/build/bin/check-env.sh
+++ b/build/bin/check-env.sh
@@ -47,3 +47,9 @@ if [ $? != 0 ]
then
quit "Failed to create $WORKING_DIR. Please make sure the user has right to access $WORKING_DIR"
fi
+
+hadoop fs -mkdir -p $WORKING_DIR/spark-history
+if [ $? != 0 ]
+then
+ quit "Failed to create $WORKING_DIR/spark-history. Please make sure the user has right to access $WORKING_DIR"
+fi
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kylin/blob/61015651/build/bin/find-spark-dependency.sh
----------------------------------------------------------------------
diff --git a/build/bin/find-spark-dependency.sh b/build/bin/find-spark-dependency.sh
new file mode 100644
index 0000000..6f74d8a
--- /dev/null
+++ b/build/bin/find-spark-dependency.sh
@@ -0,0 +1,45 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+source $(cd -P -- "$(dirname -- "$0")" && pwd -P)/header.sh
+
+spark_home=
+
+verbose Retrieving Spark dependency...
+if [ -n "$SPARK_HOME" ]
+then
+ verbose "SPARK_HOME is set to: $SPARK_HOME, use it to locate Spark dependencies."
+ spark_home=$SPARK_HOME
+fi
+
+if [ -z "$SPARK_HOME" ]
+then
+ verbose "SPARK_HOME wasn't set, use $KYLIN_HOME/spark"
+ spark_home=$KYLIN_HOME/spark
+fi
+
+spark_dependency=`find -L $spark_home -name 'spark-assembly-[a-z0-9A-Z\.-]*.jar' ! -name '*doc*' ! -name '*test*' ! -name '*sources*' ''-printf '%p:' | sed 's/:$//'`
+if [ -z "$spark_dependency" ]
+then
+ quit "spark assembly lib not found"
+else
+ verbose "spark dependency: $spark_dependency"
+ export spark_dependency
+fi
+
http://git-wip-us.apache.org/repos/asf/kylin/blob/61015651/build/bin/kylin.sh
----------------------------------------------------------------------
diff --git a/build/bin/kylin.sh b/build/bin/kylin.sh
index fca1e5c..824fb8b 100644
--- a/build/bin/kylin.sh
+++ b/build/bin/kylin.sh
@@ -45,6 +45,11 @@ function retrieveDependency() {
source ${dir}/find-kafka-dependency.sh
export HBASE_CLASSPATH=${HBASE_CLASSPATH}:${kafka_dependency}
fi
+
+ source ${dir}/find-spark-dependency.sh
+ export HBASE_CLASSPATH=${HBASE_CLASSPATH}:${spark_dependency}
+
+ verbose "HBASE_CLASSPATH: ${HBASE_CLASSPATH}"
}
# start command
@@ -112,6 +117,7 @@ then
-Dkylin.hive.dependency=${hive_dependency} \
-Dkylin.hbase.dependency=${hbase_dependency} \
-Dkylin.kafka.dependency=${kafka_dependency} \
+ -Dkylin.spark.dependency=${spark_dependency} \
-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/61015651/build/conf/kylin-spark-conf.properties
----------------------------------------------------------------------
diff --git a/build/conf/kylin-spark-conf.properties b/build/conf/kylin-spark-conf.properties
index 81567bb..5e6dafe 100644
--- a/build/conf/kylin-spark-conf.properties
+++ b/build/conf/kylin-spark-conf.properties
@@ -20,7 +20,7 @@ spark.executor.cores=4
spark.executor.instances=8
spark.history.kerberos.keytab=none
spark.history.kerberos.principal=none
-#spark.yarn.jar=hdfs://sandbox.hortonworks.com:8020/apps/spark/spark-assembly-1.6.3-hadoop2.6.0.jar
+#spark.yarn.jar=hdfs://namenode:8020/apps/spark/spark-assembly-1.6.3-hadoop2.6.0.jar
spark.driver.extraJavaOptions=-Dhdp.version=current
spark.yarn.am.extraJavaOptions=-Dhdp.version=current
spark.executor.extraJavaOptions=-Dhdp.version=current
http://git-wip-us.apache.org/repos/asf/kylin/blob/61015651/build/conf/kylin.properties
----------------------------------------------------------------------
diff --git a/build/conf/kylin.properties b/build/conf/kylin.properties
index 98b66cb..bd0bbd4 100644
--- a/build/conf/kylin.properties
+++ b/build/conf/kylin.properties
@@ -133,9 +133,6 @@ kylin.engine.mr.mapper-input-rows=1000000
# Hadoop conf folder, will export this as "HADOOP_CONF_DIR" before run spark-submit
kylin.engine.spark.env.hadoop-conf-dir=/etc/hadoop/conf
-# Spark install home, default be $KYLIN_HOME/spark/
-#kylin.engine.spark.spark-home=
-
# Spark job submission properties file, default be $KYLIN_HOME/conf/kylin-spark-conf.properties
#kylin.engine.spark.properties-file=
http://git-wip-us.apache.org/repos/asf/kylin/blob/61015651/build/script/compress.sh
----------------------------------------------------------------------
diff --git a/build/script/compress.sh b/build/script/compress.sh
index 4e3592e..39e429c 100755
--- a/build/script/compress.sh
+++ b/build/script/compress.sh
@@ -34,11 +34,12 @@ package_name=apache-kylin-${version}-bin
cd build/
rm -rf ${package_name}
mkdir ${package_name}
-cp -r lib tool bin conf tomcat ../examples/sample_cube commit_SHA1 ${package_name}
-rm -rf lib tomcat commit_SHA1
+cp -r lib tool bin conf tomcat spark ../examples/sample_cube commit_SHA1 ${package_name}
+rm -rf lib tomcat spark commit_SHA1
find ${package_name} -type d -exec chmod 755 {} \;
find ${package_name} -type f -exec chmod 644 {} \;
find ${package_name} -type f -name "*.sh" -exec chmod 755 {} \;
+find ${package_name}/spark/bin/ -type f -exec chmod +x {} \;
mkdir -p ../dist
tar -cvzf ../dist/${package_name}.tar.gz ${package_name}
rm -rf ${package_name}
http://git-wip-us.apache.org/repos/asf/kylin/blob/61015651/build/script/download-spark.sh
----------------------------------------------------------------------
diff --git a/build/script/download-spark.sh b/build/script/download-spark.sh
new file mode 100755
index 0000000..dcbcbe7
--- /dev/null
+++ b/build/script/download-spark.sh
@@ -0,0 +1,52 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+dir=$(dirname ${0})
+cd ${dir}/../..
+
+source build/script/functions.sh
+
+rm -rf build/spark
+
+spark_version="1.6.3"
+spark_pkg_md5="ce8a2e7529aac0f0175194061769dbd4"
+
+if [ ! -f "build/spark-${spark_version}-bin-hadoop2.6.tgz" ]
+then
+ echo "no binary file found"
+ wget --directory-prefix=build/ http://archive.apache.org/dist/spark/spark-${spark_version}/spark-${spark_version}-bin-hadoop2.6.tgz || echo "Download spark failed"
+else
+ if [ `calMd5 build/spark-${spark_version}-bin-hadoop2.6.tgz | awk '{print $1}'` != "${spark_pkg_md5}" ]
+ then
+ echo "md5 check failed"
+ rm build/spark-${spark_version}-bin-hadoop2.6.tgz
+ wget --directory-prefix=build/ http://archive.apache.org/dist/spark/spark-${spark_version}/spark-${spark_version}-bin-hadoop2.6.tgz || echo "Download spark failed"
+
+ fi
+fi
+
+tar -zxvf build/spark-${spark_version}-bin-hadoop2.6.tgz -C build/ || { exit 1; }
+mv build/spark-${spark_version}-bin-hadoop2.6 build/spark
+
+# Remove unused components in Spark
+rm -rf build/spark/lib/spark-examples-*
+rm -rf build/spark/examples
+rm -rf build/spark/data
+rm -rf build/spark/python
+rm -rf build/spark/R
http://git-wip-us.apache.org/repos/asf/kylin/blob/61015651/build/script/functions.sh
----------------------------------------------------------------------
diff --git a/build/script/functions.sh b/build/script/functions.sh
new file mode 100755
index 0000000..2eed617
--- /dev/null
+++ b/build/script/functions.sh
@@ -0,0 +1,60 @@
+#!/bin/bash
+
+#
+# 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.
+#
+
+function checkCommandExits() {
+ echo "Checking ${1}..."
+ if [ -z "$(command -v ${1})" ]
+ then
+ echo "Please install ${1} first so that Kylin packaging can proceed"
+ exit 1
+ else
+ echo "${1} check passed"
+ fi
+}
+
+function exportProjectVersions() {
+ if [ -z "${kylin_versoin}" ]; then
+ export kylin_version=`mvn org.apache.maven.plugins:maven-help-plugin:2.1.1:evaluate -Dexpression=project.version -f kylin | grep -Ev '(^\[|Download\w+:)'`
+ echo "Apache Kylin Version: ${kylin_version}"
+ fi
+ if [ -z "${release_version}" ]; then
+ export release_version=$kap_version
+ fi
+}
+
+function detectOSType() {
+ OS_TYPE="linux"
+ if [[ `uname -a` =~ "Darwin" ]]; then
+ OS_TYPE="mac"
+ elif [[ `uname -a` =~ "Cygwin" ]]; then
+ OS_TYPE="windows"
+ fi
+ echo $OS_TYPE
+}
+
+function calMd5() {
+ OS_TYPE=`detectOSType`
+ if [[ "$OS_TYPE" == "mac" ]]; then
+ md5 -q $1
+ elif [[ "$OS_TYPE" == "windows" ]]; then
+ md5sum $1
+ else
+ md5sum $1
+ fi
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kylin/blob/61015651/build/script/package.sh
----------------------------------------------------------------------
diff --git a/build/script/package.sh b/build/script/package.sh
index 1f9fbbd..c850ec3 100755
--- a/build/script/package.sh
+++ b/build/script/package.sh
@@ -76,6 +76,7 @@ git rev-parse HEAD >> build/commit_SHA1
sh build/script/build.sh || { exit 1; }
sh build/script/download-tomcat.sh || { exit 1; }
+sh build/script/download-spark.sh || { exit 1; }
sh build/script/prepare.sh || { exit 1; }
sh build/script/compress.sh || { exit 1; }
http://git-wip-us.apache.org/repos/asf/kylin/blob/61015651/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 f169142..4eac92a 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
@@ -187,6 +187,20 @@ public class KylinConfig extends KylinConfigBase {
return kylinConfig;
}
+ public static String getKylinConfPath() {
+ String kylinConfHome = System.getProperty(KYLIN_CONF);
+ if (!StringUtils.isEmpty(kylinConfHome)) {
+ logger.info("Use KYLIN_CONF=" + kylinConfHome);
+ return kylinConfHome;
+ }
+
+ String kylinHome = getKylinHome();
+ if (StringUtils.isEmpty(kylinHome))
+ throw new KylinConfigCannotInitException("Didn't find KYLIN_CONF or KYLIN_HOME, please set one of them");
+
+ return kylinHome + File.separator + "conf";
+ }
+
static File getKylinPropertiesFile() {
String kylinConfHome = System.getProperty(KYLIN_CONF);
if (!StringUtils.isEmpty(kylinConfHome)) {
http://git-wip-us.apache.org/repos/asf/kylin/blob/61015651/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 9f5d4db..b713f52 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
@@ -61,6 +61,16 @@ abstract public class KylinConfigBase implements Serializable {
return kylinHome;
}
+ public static String getSparkHome() {
+ String sparkHome = System.getenv("SPARK_HOME");
+ if (StringUtils.isNotEmpty(sparkHome)) {
+ logger.info("SPARK_HOME was set to " + sparkHome);
+ return sparkHome;
+ }
+
+ return getKylinHome() + File.separator + "spark";
+ }
+
// backward compatibility check happens when properties is loaded or updated
static BackwardCompatibilityConfig BCC = new BackwardCompatibilityConfig();
@@ -717,23 +727,6 @@ abstract public class KylinConfigBase implements Serializable {
// ENGINE.SPARK
// ============================================================================
- public String getSparkHome() {
- String sparkHome = getOptional("kylin.engine.spark.spark-home", "spark");
- File f = new File(sparkHome);
- if (f.exists()) {
- return f.getAbsolutePath();
- } else {
- String home = getKylinHome();
- f = new File(home, sparkHome);
- if (f.exists()) {
- return f.getAbsolutePath();
- }
- }
-
- throw new IllegalArgumentException("Spark home '" + sparkHome + "' does not exist, check 'kylin.engine.spark.spark-home' in kylin.properties");
-
- }
-
public String getSparkHadoopConfDir() {
return getRequired("kylin.engine.spark.env.hadoop-conf-dir");
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/61015651/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingJobBuilder2.java
----------------------------------------------------------------------
diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingJobBuilder2.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingJobBuilder2.java
index 9532d31..c5d47e7 100644
--- a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingJobBuilder2.java
+++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingJobBuilder2.java
@@ -19,6 +19,7 @@
package org.apache.kylin.engine.spark;
import org.apache.hadoop.util.ClassUtil;
+import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.util.StringUtil;
import org.apache.kylin.cube.CubeSegment;
import org.apache.kylin.engine.EngineFactory;
@@ -52,7 +53,7 @@ public class SparkBatchCubingJobBuilder2 extends BatchCubingJobBuilder2 {
sparkExecutable.setParam(SparkCubingByLayer.OPTION_CUBE_NAME.getOpt(), seg.getRealization().getName());
sparkExecutable.setParam(SparkCubingByLayer.OPTION_SEGMENT_ID.getOpt(), seg.getUuid());
sparkExecutable.setParam(SparkCubingByLayer.OPTION_INPUT_PATH.getOpt(), flatTableDesc.getTableName());
- sparkExecutable.setParam(SparkCubingByLayer.OPTION_CONF_PATH.getOpt(), "/Users/shishaofeng/workspace/kylin-15/examples/test_case_data/sandbox/"); //FIXME
+ sparkExecutable.setParam(SparkCubingByLayer.OPTION_CONF_PATH.getOpt(), KylinConfig.getKylinConfPath());
sparkExecutable.setParam(SparkCubingByLayer.OPTION_OUTPUT_PATH.getOpt(), cuboidRootPath);
StringBuilder jars = new StringBuilder();
@@ -65,7 +66,6 @@ public class SparkBatchCubingJobBuilder2 extends BatchCubingJobBuilder2 {
StringUtil.appendWithSeparator(jars, seg.getConfig().getSparkAdditionalJars());
sparkExecutable.setJars(jars.toString());
- // sparkExecutable.setJars("/Users/shishaofeng/.m2/repository/org/cloudera/htrace/htrace-core/2.01/htrace-core-2.01.jar,/Users/shishaofeng/.m2/repository/org/apache/hbase/hbase-protocol/0.98.8-hadoop2/hbase-protocol-0.98.8-hadoop2.jar,/Users/shishaofeng/.m2/repository/org/apache/hbase/hbase-common/0.98.8-hadoop2/hbase-common-0.98.8-hadoop2.jar,/Users/shishaofeng/.m2//repository/org/apache/hbase/hbase-client/0.98.8-hadoop2/hbase-client-0.98.8-hadoop2.jar");
sparkExecutable.setName(ExecutableConstants.STEP_NAME_BUILD_IN_MEM_CUBE + " with Spark");
return sparkExecutable;
[11/17] kylin git commit: KYLIN-2358 CuboidReducer has too many if
(aggrMask[i]) checks
Posted by sh...@apache.org.
KYLIN-2358 CuboidReducer has too many if (aggrMask[i]) checks
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/4b5257a1
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/4b5257a1
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/4b5257a1
Branch: refs/heads/sparkcubing-rebase
Commit: 4b5257a1dc4b1cf90804e5273f27944de30b8fbf
Parents: 9c98574
Author: shaofengshi <sh...@apache.org>
Authored: Thu Jan 5 13:49:08 2017 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Jan 5 13:49:08 2017 +0800
----------------------------------------------------------------------
.../kylin/measure/MeasureAggregators.java | 9 +++----
.../kylin/engine/mr/steps/CuboidReducer.java | 26 ++++++++++++--------
2 files changed, 19 insertions(+), 16 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/4b5257a1/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregators.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregators.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregators.java
index 2b28302..8951b69 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregators.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregators.java
@@ -77,13 +77,10 @@ public class MeasureAggregators implements Serializable {
}
}
- public void aggregate(Object[] values, boolean[] aggrMask) {
+ public void aggregate(Object[] values, int[] measures) {
assert values.length == descLength;
- assert aggrMask.length == descLength;
-
- for (int i = 0; i < descLength; i++) {
- if (aggrMask[i])
- aggs[i].aggregate(values[i]);
+ for (int i = 0; i < measures.length; i++) {
+ aggs[measures[i]].aggregate(values[measures[i]]);
}
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/4b5257a1/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidReducer.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidReducer.java
index afd29e3..495be77 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidReducer.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/CuboidReducer.java
@@ -22,6 +22,7 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.List;
+import com.google.common.collect.Lists;
import org.apache.hadoop.io.Text;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.cube.CubeManager;
@@ -51,7 +52,7 @@ public class CuboidReducer extends KylinReducer<Text, Text, Text, Text> {
private MeasureAggregators aggs;
private int cuboidLevel;
- private boolean[] needAggr;
+ private int[] needAggrMeasures;
private Object[] input;
private Object[] result;
private int vcounter;
@@ -76,13 +77,22 @@ public class CuboidReducer extends KylinReducer<Text, Text, Text, Text> {
input = new Object[measuresDescs.size()];
result = new Object[measuresDescs.size()];
- needAggr = new boolean[measuresDescs.size()];
- if (cuboidLevel > 0) {
- for (int i = 0; i < measuresDescs.size(); i++) {
- needAggr[i] = !measuresDescs.get(i).getFunction().getMeasureType().onlyAggrInBaseCuboid();
+ List<Integer> needAggMeasuresList = Lists.newArrayList();
+ for (int i = 0; i < measuresDescs.size(); i++) {
+ if (cuboidLevel == 0) {
+ needAggMeasuresList.add(i);
+ } else {
+ if (!measuresDescs.get(i).getFunction().getMeasureType().onlyAggrInBaseCuboid()) {
+ needAggMeasuresList.add(i);
+ }
}
}
+
+ needAggrMeasures = new int[needAggMeasuresList.size()];
+ for (int i = 0; i < needAggMeasuresList.size(); i++) {
+ needAggrMeasures[i] = needAggMeasuresList.get(i);
+ }
}
@Override
@@ -94,11 +104,7 @@ public class CuboidReducer extends KylinReducer<Text, Text, Text, Text> {
logger.info("Handling value with ordinal (This is not KV number!): " + vcounter);
}
codec.decode(ByteBuffer.wrap(value.getBytes(), 0, value.getLength()), input);
- if (cuboidLevel > 0) {
- aggs.aggregate(input, needAggr);
- } else {
- aggs.aggregate(input);
- }
+ aggs.aggregate(input, needAggrMeasures);
}
aggs.collectStates(result);
[06/17] kylin git commit: KYLIN-2297
Posted by sh...@apache.org.
KYLIN-2297
Signed-off-by: zhongjian <ji...@163.com>
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/7e71df94
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/7e71df94
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/7e71df94
Branch: refs/heads/sparkcubing-rebase
Commit: 7e71df9479ebc82b39211945a83f3147ddb70592
Parents: 89e4b53
Author: luguosheng <55...@qq.com>
Authored: Thu Dec 29 16:54:15 2016 +0800
Committer: zhongjian <ji...@163.com>
Committed: Thu Jan 5 11:27:13 2017 +0800
----------------------------------------------------------------------
webapp/app/js/directives/directives.js | 11 ++++++++---
1 file changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/7e71df94/webapp/app/js/directives/directives.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/directives/directives.js b/webapp/app/js/directives/directives.js
index 3d18451..ca08493 100644
--- a/webapp/app/js/directives/directives.js
+++ b/webapp/app/js/directives/directives.js
@@ -278,12 +278,17 @@ KylinApp.directive('kylinPagination', function ($parse, $q) {
});
ctrl.$parsers.push(function (value) {
- if(/\d{4}-\d{1,2}-\d{1,2}\s+(\d{1,2}:\d{1,2}:\d{1,2})/.test(value)) {
- value=new Date(value);
+ var date;
+ if(/^\d{4}-\d{1,2}-\d{1,2}(\s+\d{1,2}:\d{1,2}:\d{1,2})?$/.test(value)) {
+ date=new Date(value);
+ if(!date||date&&!date.getTime()){
+ return value;
+ }else{
+ return date.getTime()-(60000 * date.getTimezoneOffset());
+ }
}else{
return value;
}
- return value.getTime()-(60000 * value.getTimezoneOffset());
});
}
};
[17/17] kylin git commit: KYLIN-2331 use org.apache.htrace.Trace to
find htrace-core.jar
Posted by sh...@apache.org.
KYLIN-2331 use org.apache.htrace.Trace to find htrace-core.jar
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/afd1ac22
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/afd1ac22
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/afd1ac22
Branch: refs/heads/sparkcubing-rebase
Commit: afd1ac22346571b39108d3a891bebec80e0ecaba
Parents: 6101565
Author: shaofengshi <sh...@apache.org>
Authored: Tue Jan 3 13:36:49 2017 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Jan 5 15:20:48 2017 +0800
----------------------------------------------------------------------
.../apache/kylin/engine/spark/SparkBatchCubingJobBuilder2.java | 4 +++-
1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/afd1ac22/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingJobBuilder2.java
----------------------------------------------------------------------
diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingJobBuilder2.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingJobBuilder2.java
index c5d47e7..9431468 100644
--- a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingJobBuilder2.java
+++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingJobBuilder2.java
@@ -59,10 +59,12 @@ public class SparkBatchCubingJobBuilder2 extends BatchCubingJobBuilder2 {
StringBuilder jars = new StringBuilder();
StringUtil.appendWithSeparator(jars, findJar("org.htrace.HTraceConfiguration")); // htrace-core.jar
- StringUtil.appendWithSeparator(jars, findJar("org.cloudera.htrace.HTraceConfiguration"));
+ StringUtil.appendWithSeparator(jars, findJar("org.apache.htrace.Trace")); // htrace-core.jar
+ StringUtil.appendWithSeparator(jars, findJar("org.cloudera.htrace.HTraceConfiguration")); // htrace-core.jar
StringUtil.appendWithSeparator(jars, findJar("org.apache.hadoop.hbase.client.HConnection")); // hbase-client.jar
StringUtil.appendWithSeparator(jars, findJar("org.apache.hadoop.hbase.HBaseConfiguration")); // hbase-common.jar
StringUtil.appendWithSeparator(jars, findJar("org.apache.hadoop.hbase.util.ByteStringer")); // hbase-protocol.jar
+ StringUtil.appendWithSeparator(jars, findJar("com.yammer.metrics.core.Gauge")); // metrics-core.jar
StringUtil.appendWithSeparator(jars, seg.getConfig().getSparkAdditionalJars());
sparkExecutable.setJars(jars.toString());
[13/17] kylin git commit: KYLIN-2331 by layer spark cubing
Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BaseCuboidBuilder.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BaseCuboidBuilder.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BaseCuboidBuilder.java
new file mode 100644
index 0000000..07b636b
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/BaseCuboidBuilder.java
@@ -0,0 +1,173 @@
+/*
+ * 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.engine.mr.common;
+
+import com.google.common.collect.Sets;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.Dictionary;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.kv.AbstractRowKeyEncoder;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.CubeJoinedFlatTableEnrich;
+import org.apache.kylin.measure.BufferedMeasureCodec;
+import org.apache.kylin.measure.MeasureIngester;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.kylin.metadata.model.ParameterDesc;
+import org.apache.kylin.metadata.model.TblColRef;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ */
+public class BaseCuboidBuilder implements java.io.Serializable {
+
+ protected static final Logger logger = LoggerFactory.getLogger(BaseCuboidBuilder.class);
+ public static final String HIVE_NULL = "\\N";
+ protected String cubeName;
+ protected Cuboid baseCuboid;
+ protected CubeDesc cubeDesc;
+ protected CubeSegment cubeSegment;
+ protected Set<String> nullStrs;
+ protected CubeJoinedFlatTableEnrich intermediateTableDesc;
+ protected MeasureIngester<?>[] aggrIngesters;
+ protected Map<TblColRef, Dictionary<String>> dictionaryMap;
+ protected AbstractRowKeyEncoder rowKeyEncoder;
+ protected BufferedMeasureCodec measureCodec;
+
+ protected KylinConfig kylinConfig;
+
+ public BaseCuboidBuilder(KylinConfig kylinConfig, CubeDesc cubeDesc, CubeSegment cubeSegment, CubeJoinedFlatTableEnrich intermediateTableDesc,
+ AbstractRowKeyEncoder rowKeyEncoder, MeasureIngester<?>[] aggrIngesters, Map<TblColRef, Dictionary<String>> dictionaryMap) {
+ this.kylinConfig = kylinConfig;
+ this.cubeDesc = cubeDesc;
+ this.cubeSegment = cubeSegment;
+ this.intermediateTableDesc = intermediateTableDesc;
+ this.rowKeyEncoder = rowKeyEncoder;
+ this.aggrIngesters = aggrIngesters;
+ this.dictionaryMap = dictionaryMap;
+
+ init();
+ measureCodec = new BufferedMeasureCodec(cubeDesc.getMeasures());
+ }
+
+ public BaseCuboidBuilder(KylinConfig kylinConfig, CubeDesc cubeDesc, CubeSegment cubeSegment, CubeJoinedFlatTableEnrich intermediateTableDesc) {
+ this.kylinConfig = kylinConfig;
+ this.cubeDesc = cubeDesc;
+ this.cubeSegment = cubeSegment;
+ this.intermediateTableDesc = intermediateTableDesc;
+
+ init();
+ rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cubeSegment, baseCuboid);
+ measureCodec = new BufferedMeasureCodec(cubeDesc.getMeasures());
+ aggrIngesters = MeasureIngester.create(cubeDesc.getMeasures());
+ dictionaryMap = cubeSegment.buildDictionaryMap();
+
+ }
+
+ private void init() {
+ long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+ baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+ initNullBytes();
+ }
+
+ private void initNullBytes() {
+ nullStrs = Sets.newHashSet();
+ nullStrs.add(HIVE_NULL);
+ String[] nullStrings = cubeDesc.getNullStrings();
+ if (nullStrings != null) {
+ for (String s : nullStrings) {
+ nullStrs.add(s);
+ }
+ }
+ }
+
+ protected boolean isNull(String v) {
+ return nullStrs.contains(v);
+ }
+
+ public byte[] buildKey(String[] flatRow) {
+ int[] rowKeyColumnIndexes = intermediateTableDesc.getRowKeyColumnIndexes();
+ List<TblColRef> columns = baseCuboid.getColumns();
+ String[] colValues = new String[columns.size()];
+ for (int i = 0; i < columns.size(); i++) {
+ colValues[i] = getCell(rowKeyColumnIndexes[i], flatRow);
+ }
+ return rowKeyEncoder.encode(colValues);
+ }
+
+ public ByteBuffer buildValue(String[] flatRow) {
+ return measureCodec.encode(buildValueObjects(flatRow));
+ }
+
+ public Object[] buildValueObjects(String[] flatRow) {
+ Object[] measures = new Object[cubeDesc.getMeasures().size()];
+ for (int i = 0; i < measures.length; i++) {
+ measures[i] = buildValueOf(i, flatRow);
+ }
+
+ return measures;
+ }
+
+ public void resetAggrs() {
+ for (int i = 0; i < cubeDesc.getMeasures().size(); i++) {
+ aggrIngesters[i].reset();
+ }
+ }
+
+ private Object buildValueOf(int idxOfMeasure, String[] flatRow) {
+ MeasureDesc measure = cubeDesc.getMeasures().get(idxOfMeasure);
+ FunctionDesc function = measure.getFunction();
+ int[] colIdxOnFlatTable = intermediateTableDesc.getMeasureColumnIndexes()[idxOfMeasure];
+
+ int paramCount = function.getParameterCount();
+ String[] inputToMeasure = new String[paramCount];
+
+ // pick up parameter values
+ ParameterDesc param = function.getParameter();
+ int colParamIdx = 0; // index among parameters of column type
+ for (int i = 0; i < paramCount; i++, param = param.getNextParameter()) {
+ String value;
+ if (function.isCount()) {
+ value = "1";
+ } else if (param.isColumnType()) {
+ value = getCell(colIdxOnFlatTable[colParamIdx++], flatRow);
+ } else {
+ value = param.getValue();
+ }
+ inputToMeasure[i] = value;
+ }
+
+ return aggrIngesters[idxOfMeasure].valueOf(inputToMeasure, measure, dictionaryMap);
+ }
+
+ private String getCell(int i, String[] flatRow) {
+ if (isNull(flatRow[i]))
+ return null;
+ else
+ return flatRow[i];
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/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 ffba181..4011915 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
@@ -262,6 +262,11 @@ public class CubeStatsReader {
return ret;
}
+ public List<Long> getCuboidsByLayer(int level) {
+ List<List<Long>> layeredCuboids = cuboidScheduler.getCuboidsByLayer();
+ return layeredCuboids.get(level);
+ }
+
private void printCuboidInfoTreeEntry(Map<Long, Long> cuboidRows, Map<Long, Double> cuboidSizes, PrintWriter out) {
long baseCuboid = Cuboid.getBaseCuboidId(seg.getCubeDesc());
int dimensionCount = Long.bitCount(baseCuboid);
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/NDCuboidBuilder.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/NDCuboidBuilder.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/NDCuboidBuilder.java
new file mode 100644
index 0000000..4e98618
--- /dev/null
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/common/NDCuboidBuilder.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.engine.mr.common;
+
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.common.util.SplittedBytes;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.common.RowKeySplitter;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.kv.RowConstants;
+import org.apache.kylin.cube.kv.RowKeyEncoder;
+import org.apache.kylin.cube.kv.RowKeyEncoderProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+
+/**
+ */
+public class NDCuboidBuilder implements Serializable {
+
+ protected static final Logger logger = LoggerFactory.getLogger(NDCuboidBuilder.class);
+ protected String cubeName;
+ protected String segmentID;
+ protected CubeSegment cubeSegment;
+ private RowKeySplitter rowKeySplitter;
+ private RowKeyEncoderProvider rowKeyEncoderProvider;
+ private byte[] newKeyBodyBuf = new byte[RowConstants.ROWKEY_BUFFER_SIZE];
+ private ByteArray newKeyBuf = ByteArray.allocate(RowConstants.ROWKEY_BUFFER_SIZE);
+
+ public NDCuboidBuilder(CubeSegment cubeSegment) {
+ this.cubeSegment = cubeSegment;
+ this.rowKeySplitter = new RowKeySplitter(cubeSegment, 65, 256);
+ this.rowKeyEncoderProvider = new RowKeyEncoderProvider(cubeSegment);
+ }
+
+ public NDCuboidBuilder(CubeSegment cubeSegment, RowKeyEncoderProvider rowKeyEncoderProvider) {
+ this.cubeSegment = cubeSegment;
+ this.rowKeyEncoderProvider = rowKeyEncoderProvider;
+ this.rowKeySplitter = new RowKeySplitter(cubeSegment, 65, 256);
+ }
+
+
+ public Pair<Integer, ByteArray> buildKey(Cuboid parentCuboid, Cuboid childCuboid, SplittedBytes[] splitBuffers) {
+ RowKeyEncoder rowkeyEncoder = rowKeyEncoderProvider.getRowkeyEncoder(childCuboid);
+
+ int offset = 0;
+
+ // rowkey columns
+ long mask = Long.highestOneBit(parentCuboid.getId());
+ long parentCuboidId = parentCuboid.getId();
+ long childCuboidId = childCuboid.getId();
+ long parentCuboidIdActualLength = Long.SIZE - Long.numberOfLeadingZeros(parentCuboid.getId());
+ int index = rowKeySplitter.getBodySplitOffset(); // skip shard and cuboidId
+ for (int i = 0; i < parentCuboidIdActualLength; i++) {
+ if ((mask & parentCuboidId) > 0) {// if the this bit position equals
+ // 1
+ if ((mask & childCuboidId) > 0) {// if the child cuboid has this
+ // column
+ System.arraycopy(splitBuffers[index].value, 0, newKeyBodyBuf, offset, splitBuffers[index].length);
+ offset += splitBuffers[index].length;
+ }
+ index++;
+ }
+ mask = mask >> 1;
+ }
+
+ int fullKeySize = rowkeyEncoder.getBytesLength();
+ while (newKeyBuf.array().length < fullKeySize) {
+ newKeyBuf.set(new byte[newKeyBuf.length() * 2]);
+ }
+ newKeyBuf.set(0, fullKeySize);
+
+ rowkeyEncoder.encode(new ByteArray(newKeyBodyBuf, 0, offset), newKeyBuf);
+
+ return new Pair<>(Integer.valueOf(fullKeySize), newKeyBuf);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
index 7b719e0..d08e29a 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/BaseCuboidMapperBase.java
@@ -18,38 +18,25 @@
package org.apache.kylin.engine.mr.steps;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
import org.apache.hadoop.io.Text;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.common.util.Dictionary;
import org.apache.kylin.cube.CubeInstance;
import org.apache.kylin.cube.CubeManager;
import org.apache.kylin.cube.CubeSegment;
-import org.apache.kylin.cube.cuboid.Cuboid;
-import org.apache.kylin.cube.kv.AbstractRowKeyEncoder;
import org.apache.kylin.cube.model.CubeDesc;
import org.apache.kylin.cube.model.CubeJoinedFlatTableEnrich;
import org.apache.kylin.engine.EngineFactory;
import org.apache.kylin.engine.mr.KylinMapper;
import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
+import org.apache.kylin.engine.mr.common.BaseCuboidBuilder;
import org.apache.kylin.engine.mr.common.BatchConstants;
-import org.apache.kylin.measure.BufferedMeasureCodec;
-import org.apache.kylin.measure.MeasureIngester;
-import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.MeasureDesc;
-import org.apache.kylin.metadata.model.ParameterDesc;
-import org.apache.kylin.metadata.model.TblColRef;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import com.google.common.collect.Sets;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
/**
*/
@@ -59,131 +46,37 @@ abstract public class BaseCuboidMapperBase<KEYIN, VALUEIN> extends KylinMapper<K
public static final byte[] ONE = Bytes.toBytes("1");
protected String cubeName;
protected String segmentID;
- protected Cuboid baseCuboid;
protected CubeInstance cube;
protected CubeDesc cubeDesc;
protected CubeSegment cubeSegment;
- protected Set<String> nullStrs;
- protected CubeJoinedFlatTableEnrich intermediateTableDesc;
- protected String intermediateTableRowDelimiter;
- protected byte byteRowDelimiter;
protected int counter;
- protected MeasureIngester<?>[] aggrIngesters;
- protected Map<TblColRef, Dictionary<String>> dictionaryMap;
protected Object[] measures;
- protected AbstractRowKeyEncoder rowKeyEncoder;
- protected BufferedMeasureCodec measureCodec;
private int errorRecordCounter;
protected Text outputKey = new Text();
protected Text outputValue = new Text();
+ private BaseCuboidBuilder baseCuboidBuilder;
+
@Override
protected void setup(Context context) throws IOException {
super.bindCurrentConfiguration(context.getConfiguration());
-
cubeName = context.getConfiguration().get(BatchConstants.CFG_CUBE_NAME).toUpperCase();
segmentID = context.getConfiguration().get(BatchConstants.CFG_CUBE_SEGMENT_ID);
- intermediateTableRowDelimiter = context.getConfiguration().get(BatchConstants.CFG_CUBE_INTERMEDIATE_TABLE_ROW_DELIMITER, Character.toString(BatchConstants.INTERMEDIATE_TABLE_ROW_DELIMITER));
- if (Bytes.toBytes(intermediateTableRowDelimiter).length > 1) {
- throw new RuntimeException("Expected delimiter byte length is 1, but got " + Bytes.toBytes(intermediateTableRowDelimiter).length);
- }
-
- byteRowDelimiter = Bytes.toBytes(intermediateTableRowDelimiter)[0];
-
- KylinConfig config = AbstractHadoopJob.loadKylinPropsAndMetadata();
-
- cube = CubeManager.getInstance(config).getCube(cubeName);
+ final KylinConfig kylinConfig = AbstractHadoopJob.loadKylinPropsAndMetadata();
+ cube = CubeManager.getInstance(kylinConfig).getCube(cubeName);
cubeDesc = cube.getDescriptor();
cubeSegment = cube.getSegmentById(segmentID);
+ CubeJoinedFlatTableEnrich intermediateTableDesc = new CubeJoinedFlatTableEnrich(EngineFactory.getJoinedFlatTableDesc(cubeSegment), cubeDesc);
+ baseCuboidBuilder = new BaseCuboidBuilder(kylinConfig, cubeDesc, cubeSegment, intermediateTableDesc);
- long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
- baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
-
- intermediateTableDesc = new CubeJoinedFlatTableEnrich(EngineFactory.getJoinedFlatTableDesc(cubeSegment), cubeDesc);
-
- rowKeyEncoder = AbstractRowKeyEncoder.createInstance(cubeSegment, baseCuboid);
-
- measureCodec = new BufferedMeasureCodec(cubeDesc.getMeasures());
- measures = new Object[cubeDesc.getMeasures().size()];
-
- aggrIngesters = MeasureIngester.create(cubeDesc.getMeasures());
- dictionaryMap = cubeSegment.buildDictionaryMap();
-
- initNullBytes();
- }
-
- private void initNullBytes() {
- nullStrs = Sets.newHashSet();
- nullStrs.add(HIVE_NULL);
- String[] nullStrings = cubeDesc.getNullStrings();
- if (nullStrings != null) {
- for (String s : nullStrings) {
- nullStrs.add(s);
- }
- }
- }
-
- protected boolean isNull(String v) {
- return nullStrs.contains(v);
- }
-
- protected byte[] buildKey(String[] flatRow) {
- int[] rowKeyColumnIndexes = intermediateTableDesc.getRowKeyColumnIndexes();
- List<TblColRef> columns = baseCuboid.getColumns();
- String[] colValues = new String[columns.size()];
- for (int i = 0; i < columns.size(); i++) {
- colValues[i] = getCell(rowKeyColumnIndexes[i], flatRow);
- }
- return rowKeyEncoder.encode(colValues);
}
- private ByteBuffer buildValue(String[] flatRow) {
-
- for (int i = 0; i < measures.length; i++) {
- measures[i] = buildValueOf(i, flatRow);
- }
-
- return measureCodec.encode(measures);
- }
-
- private Object buildValueOf(int idxOfMeasure, String[] flatRow) {
- MeasureDesc measure = cubeDesc.getMeasures().get(idxOfMeasure);
- FunctionDesc function = measure.getFunction();
- int[] colIdxOnFlatTable = intermediateTableDesc.getMeasureColumnIndexes()[idxOfMeasure];
-
- int paramCount = function.getParameterCount();
- String[] inputToMeasure = new String[paramCount];
-
- // pick up parameter values
- ParameterDesc param = function.getParameter();
- int colParamIdx = 0; // index among parameters of column type
- for (int i = 0; i < paramCount; i++, param = param.getNextParameter()) {
- String value;
- if (function.isCount()) {
- value = "1";
- } else if (param.isColumnType()) {
- value = getCell(colIdxOnFlatTable[colParamIdx++], flatRow);
- } else {
- value = param.getValue();
- }
- inputToMeasure[i] = value;
- }
-
- return aggrIngesters[idxOfMeasure].valueOf(inputToMeasure, measure, dictionaryMap);
- }
-
- private String getCell(int i, String[] flatRow) {
- if (isNull(flatRow[i]))
- return null;
- else
- return flatRow[i];
- }
protected void outputKV(String[] flatRow, Context context) throws IOException, InterruptedException {
- byte[] rowKey = buildKey(flatRow);
+ byte[] rowKey = baseCuboidBuilder.buildKey(flatRow);
outputKey.set(rowKey, 0, rowKey.length);
- ByteBuffer valueBuf = buildValue(flatRow);
+ ByteBuffer valueBuf = baseCuboidBuilder.buildValue(flatRow);
outputValue.set(valueBuf.array(), 0, valueBuf.position());
context.write(outputKey, outputValue);
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
index 01cdd4a..b924edc 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/NDCuboidMapper.java
@@ -18,29 +18,27 @@
package org.apache.kylin.engine.mr.steps;
-import java.io.IOException;
-import java.util.Collection;
-
import org.apache.hadoop.io.Text;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.util.ByteArray;
-import org.apache.kylin.common.util.SplittedBytes;
+import org.apache.kylin.common.util.Pair;
import org.apache.kylin.cube.CubeInstance;
import org.apache.kylin.cube.CubeManager;
import org.apache.kylin.cube.CubeSegment;
import org.apache.kylin.cube.common.RowKeySplitter;
import org.apache.kylin.cube.cuboid.Cuboid;
import org.apache.kylin.cube.cuboid.CuboidScheduler;
-import org.apache.kylin.cube.kv.RowConstants;
-import org.apache.kylin.cube.kv.RowKeyEncoder;
-import org.apache.kylin.cube.kv.RowKeyEncoderProvider;
import org.apache.kylin.cube.model.CubeDesc;
import org.apache.kylin.engine.mr.KylinMapper;
import org.apache.kylin.engine.mr.common.AbstractHadoopJob;
import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.common.NDCuboidBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.io.IOException;
+import java.util.Collection;
+
/**
* @author George Song (ysong1)
*
@@ -59,10 +57,9 @@ public class NDCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
private int handleCounter;
private int skipCounter;
- private byte[] newKeyBodyBuf = new byte[RowConstants.ROWKEY_BUFFER_SIZE];
- private ByteArray newKeyBuf = ByteArray.allocate(RowConstants.ROWKEY_BUFFER_SIZE);
private RowKeySplitter rowKeySplitter;
- private RowKeyEncoderProvider rowKeyEncoderProvider;
+
+ private NDCuboidBuilder ndCuboidBuilder;
@Override
protected void setup(Context context) throws IOException {
@@ -76,48 +73,13 @@ public class NDCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
CubeInstance cube = CubeManager.getInstance(config).getCube(cubeName);
cubeSegment = cube.getSegmentById(segmentID);
cubeDesc = cube.getDescriptor();
-
+ ndCuboidBuilder = new NDCuboidBuilder(cubeSegment);
// initialize CubiodScheduler
cuboidScheduler = new CuboidScheduler(cubeDesc);
-
rowKeySplitter = new RowKeySplitter(cubeSegment, 65, 256);
- rowKeyEncoderProvider = new RowKeyEncoderProvider(cubeSegment);
}
- private int buildKey(Cuboid parentCuboid, Cuboid childCuboid, SplittedBytes[] splitBuffers) {
- RowKeyEncoder rowkeyEncoder = rowKeyEncoderProvider.getRowkeyEncoder(childCuboid);
-
- int offset = 0;
-
- // rowkey columns
- long mask = Long.highestOneBit(parentCuboid.getId());
- long parentCuboidId = parentCuboid.getId();
- long childCuboidId = childCuboid.getId();
- long parentCuboidIdActualLength = Long.SIZE - Long.numberOfLeadingZeros(parentCuboid.getId());
- int index = rowKeySplitter.getBodySplitOffset(); // skip shard and cuboidId
- for (int i = 0; i < parentCuboidIdActualLength; i++) {
- if ((mask & parentCuboidId) > 0) {// if the this bit position equals
- // 1
- if ((mask & childCuboidId) > 0) {// if the child cuboid has this
- // column
- System.arraycopy(splitBuffers[index].value, 0, newKeyBodyBuf, offset, splitBuffers[index].length);
- offset += splitBuffers[index].length;
- }
- index++;
- }
- mask = mask >> 1;
- }
- int fullKeySize = rowkeyEncoder.getBytesLength();
- while (newKeyBuf.array().length < fullKeySize) {
- newKeyBuf.set(new byte[newKeyBuf.length() * 2]);
- }
- newKeyBuf.set(0, fullKeySize);
-
- rowkeyEncoder.encode(new ByteArray(newKeyBodyBuf, 0, offset), newKeyBuf);
-
- return fullKeySize;
- }
@Override
public void doMap(Text key, Text value, Context context) throws IOException, InterruptedException {
@@ -143,8 +105,8 @@ public class NDCuboidMapper extends KylinMapper<Text, Text, Text, Text> {
for (Long child : myChildren) {
Cuboid childCuboid = Cuboid.findById(cubeDesc, child);
- int fullKeySize = buildKey(parentCuboid, childCuboid, rowKeySplitter.getSplitBuffers());
- outputKey.set(newKeyBuf.array(), 0, fullKeySize);
+ Pair<Integer, ByteArray> result = ndCuboidBuilder.buildKey(parentCuboid, childCuboid, rowKeySplitter.getSplitBuffers());
+ outputKey.set(result.getSecond().array(), 0, result.getFirst());
context.write(outputKey, value);
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeReducerTest.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeReducerTest.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeReducerTest.java
index 97dd750..29ca9b8 100644
--- a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeReducerTest.java
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeReducerTest.java
@@ -41,7 +41,6 @@ import org.apache.kylin.measure.BufferedMeasureCodec;
import org.apache.kylin.measure.MeasureAggregator;
import org.apache.kylin.measure.MeasureIngester;
import org.apache.kylin.measure.MeasureType;
-import org.apache.kylin.metadata.datatype.LongMutable;
import org.apache.kylin.metadata.model.FunctionDesc;
import org.apache.kylin.metadata.model.MeasureDesc;
import org.junit.After;
@@ -161,7 +160,7 @@ public class CubeReducerTest extends LocalFileMetadataTestCase {
}
private Text newValueText(BufferedMeasureCodec codec, String sum, String min, String max, int count, int item_count) {
- Object[] values = new Object[] { new BigDecimal(sum), new BigDecimal(min), new BigDecimal(max), new LongMutable(count), new LongMutable(item_count) };
+ Object[] values = new Object[] { new BigDecimal(sum), new BigDecimal(min), new BigDecimal(max), new Long(count), new Long(item_count) };
ByteBuffer buf = codec.encode(values);
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/engine-spark/pom.xml
----------------------------------------------------------------------
diff --git a/engine-spark/pom.xml b/engine-spark/pom.xml
index 504a966..a931fac 100644
--- a/engine-spark/pom.xml
+++ b/engine-spark/pom.xml
@@ -47,6 +47,11 @@
<artifactId>kylin-core-job</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.apache.kylin</groupId>
+ <artifactId>kylin-engine-mr</artifactId>
+ </dependency>
+
<!-- Spark dependency -->
<dependency>
<groupId>org.apache.spark</groupId>
@@ -115,6 +120,10 @@
<artifactId>maven-model</artifactId>
<scope>test</scope>
</dependency>
+ <dependency>
+ <groupId>com.google.guava</groupId>
+ <artifactId>guava</artifactId>
+ </dependency>
</dependencies>
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingEngine2.java
----------------------------------------------------------------------
diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingEngine2.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingEngine2.java
new file mode 100644
index 0000000..a7a4151
--- /dev/null
+++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingEngine2.java
@@ -0,0 +1,33 @@
+/*
+ * 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.engine.spark;
+
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.mr.MRBatchCubingEngine2;
+import org.apache.kylin.job.execution.DefaultChainedExecutable;
+
+/**
+ */
+public class SparkBatchCubingEngine2 extends MRBatchCubingEngine2 {
+ @Override
+ public DefaultChainedExecutable createBatchCubingJob(CubeSegment newSegment, String submitter) {
+ return new SparkBatchCubingJobBuilder2(newSegment, submitter).build();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingJobBuilder2.java
----------------------------------------------------------------------
diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingJobBuilder2.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingJobBuilder2.java
new file mode 100644
index 0000000..9532d31
--- /dev/null
+++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkBatchCubingJobBuilder2.java
@@ -0,0 +1,85 @@
+/*
+ * 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.engine.spark;
+
+import org.apache.hadoop.util.ClassUtil;
+import org.apache.kylin.common.util.StringUtil;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.engine.EngineFactory;
+import org.apache.kylin.engine.mr.BatchCubingJobBuilder2;
+import org.apache.kylin.engine.mr.CubingJob;
+import org.apache.kylin.job.constant.ExecutableConstants;
+import org.apache.kylin.job.execution.AbstractExecutable;
+import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ */
+public class SparkBatchCubingJobBuilder2 extends BatchCubingJobBuilder2 {
+
+ private static final Logger logger = LoggerFactory.getLogger(SparkBatchCubingJobBuilder2.class);
+
+ public SparkBatchCubingJobBuilder2(CubeSegment newSegment, String submitter) {
+ super(newSegment, submitter);
+ }
+
+ protected void addLayerCubingSteps(final CubingJob result, final String jobId, final String cuboidRootPath) {
+
+ }
+
+ @Override
+ protected AbstractExecutable createInMemCubingStep(String jobId, String cuboidRootPath) {
+ IJoinedFlatTableDesc flatTableDesc = EngineFactory.getJoinedFlatTableDesc(seg);
+ final SparkExecutable sparkExecutable = new SparkExecutable();
+ sparkExecutable.setClassName(SparkCubingByLayer.class.getName());
+ sparkExecutable.setParam(SparkCubingByLayer.OPTION_CUBE_NAME.getOpt(), seg.getRealization().getName());
+ sparkExecutable.setParam(SparkCubingByLayer.OPTION_SEGMENT_ID.getOpt(), seg.getUuid());
+ sparkExecutable.setParam(SparkCubingByLayer.OPTION_INPUT_PATH.getOpt(), flatTableDesc.getTableName());
+ sparkExecutable.setParam(SparkCubingByLayer.OPTION_CONF_PATH.getOpt(), "/Users/shishaofeng/workspace/kylin-15/examples/test_case_data/sandbox/"); //FIXME
+ sparkExecutable.setParam(SparkCubingByLayer.OPTION_OUTPUT_PATH.getOpt(), cuboidRootPath);
+
+ StringBuilder jars = new StringBuilder();
+
+ StringUtil.appendWithSeparator(jars, findJar("org.htrace.HTraceConfiguration")); // htrace-core.jar
+ StringUtil.appendWithSeparator(jars, findJar("org.cloudera.htrace.HTraceConfiguration"));
+ StringUtil.appendWithSeparator(jars, findJar("org.apache.hadoop.hbase.client.HConnection")); // hbase-client.jar
+ StringUtil.appendWithSeparator(jars, findJar("org.apache.hadoop.hbase.HBaseConfiguration")); // hbase-common.jar
+ StringUtil.appendWithSeparator(jars, findJar("org.apache.hadoop.hbase.util.ByteStringer")); // hbase-protocol.jar
+
+ StringUtil.appendWithSeparator(jars, seg.getConfig().getSparkAdditionalJars());
+ sparkExecutable.setJars(jars.toString());
+ // sparkExecutable.setJars("/Users/shishaofeng/.m2/repository/org/cloudera/htrace/htrace-core/2.01/htrace-core-2.01.jar,/Users/shishaofeng/.m2/repository/org/apache/hbase/hbase-protocol/0.98.8-hadoop2/hbase-protocol-0.98.8-hadoop2.jar,/Users/shishaofeng/.m2/repository/org/apache/hbase/hbase-common/0.98.8-hadoop2/hbase-common-0.98.8-hadoop2.jar,/Users/shishaofeng/.m2//repository/org/apache/hbase/hbase-client/0.98.8-hadoop2/hbase-client-0.98.8-hadoop2.jar");
+
+ sparkExecutable.setName(ExecutableConstants.STEP_NAME_BUILD_IN_MEM_CUBE + " with Spark");
+ return sparkExecutable;
+
+ }
+
+ private String findJar(String className) {
+ try {
+ return ClassUtil.findContainingJar(Class.forName(className));
+ } catch (ClassNotFoundException e) {
+ logger.error("failed to locate jar for class " + className, e);
+ }
+
+ return "";
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java
----------------------------------------------------------------------
diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java
index 6e894dd..f06c338 100644
--- a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java
+++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubing.java
@@ -84,6 +84,7 @@ import org.apache.kylin.engine.spark.util.IteratorUtils;
import org.apache.kylin.measure.BufferedMeasureCodec;
import org.apache.kylin.measure.MeasureAggregators;
import org.apache.kylin.measure.hllc.HLLCounter;
+import org.apache.kylin.measure.MeasureIngester;
import org.apache.kylin.metadata.model.FunctionDesc;
import org.apache.kylin.metadata.model.IJoinedFlatTableDesc;
import org.apache.kylin.metadata.model.MeasureDesc;
@@ -153,6 +154,20 @@ public class SparkCubing extends AbstractApplication {
return options;
}
+ public static KylinConfig loadKylinPropsAndMetadata(String folder) throws IOException {
+ File metaDir = new File(folder);
+ if (!metaDir.getAbsolutePath().equals(System.getProperty(KylinConfig.KYLIN_CONF))) {
+ System.setProperty(KylinConfig.KYLIN_CONF, metaDir.getAbsolutePath());
+ logger.info("The absolute path for meta dir is " + metaDir.getAbsolutePath());
+ KylinConfig kylinConfig = KylinConfig.getInstanceFromEnv();
+ System.out.println("setting metadataUrl to " + metaDir.getAbsolutePath());
+ kylinConfig.setMetadataUrl(metaDir.getAbsolutePath());
+ return kylinConfig;
+ } else {
+ return KylinConfig.getInstanceFromEnv();
+ }
+ }
+
private void setupClasspath(JavaSparkContext sc, String confPath) throws Exception {
ClassUtil.addClasspath(confPath);
final File[] files = new File(confPath).listFiles(new FileFilter() {
@@ -462,7 +477,7 @@ public class SparkCubing extends AbstractApplication {
}).saveAsNewAPIHadoopFile(hFileLocation, ImmutableBytesWritable.class, KeyValue.class, HFileOutputFormat.class, conf);
}
- private static void prepare() throws Exception {
+ public static void prepare() throws Exception {
final File file = new File(SparkFiles.get("kylin.properties"));
final String confPath = file.getParentFile().getAbsolutePath();
System.out.println("conf directory:" + confPath);
@@ -526,12 +541,18 @@ public class SparkCubing extends AbstractApplication {
}
}
- private Collection<String> getKyroClasses() {
+ public static Collection<String> getKyroClasses() {
Set<Class> kyroClasses = Sets.newHashSet();
kyroClasses.addAll(new Reflections("org.apache.kylin").getSubTypesOf(Serializable.class));
+ kyroClasses.addAll(new Reflections("org.apache.kylin.dimension").getSubTypesOf(Serializable.class));
+ kyroClasses.addAll(new Reflections("org.apache.kylin.cube").getSubTypesOf(Serializable.class));
kyroClasses.addAll(new Reflections("org.apache.kylin.cube.model").getSubTypesOf(Object.class));
+ kyroClasses.addAll(new Reflections("org.apache.kylin.metadata").getSubTypesOf(Object.class));
kyroClasses.addAll(new Reflections("org.apache.kylin.metadata.model").getSubTypesOf(Object.class));
kyroClasses.addAll(new Reflections("org.apache.kylin.metadata.measure").getSubTypesOf(Object.class));
+ kyroClasses.addAll(new Reflections("org.apache.kylin.metadata.datatype").getSubTypesOf(org.apache.kylin.common.util.BytesSerializer.class));
+ kyroClasses.addAll(new Reflections("org.apache.kylin.measure").getSubTypesOf(MeasureIngester.class));
+
kyroClasses.add(HashMap.class);
kyroClasses.add(org.apache.spark.sql.Row[].class);
kyroClasses.add(org.apache.spark.sql.Row.class);
@@ -541,11 +562,15 @@ public class SparkCubing extends AbstractApplication {
kyroClasses.add(org.apache.spark.sql.types.StructField.class);
kyroClasses.add(org.apache.spark.sql.types.DateType$.class);
kyroClasses.add(org.apache.spark.sql.types.Metadata.class);
- kyroClasses.add(Object[].class);
kyroClasses.add(org.apache.spark.sql.types.StringType$.class);
kyroClasses.add(Hashing.murmur3_128().getClass());
- kyroClasses.add(org.apache.spark.sql.columnar.CachedBatch.class);
+ kyroClasses.add(org.apache.spark.sql.execution.columnar.CachedBatch.class);
+ kyroClasses.add(Object[].class);
+ kyroClasses.add(int[].class);
+ kyroClasses.add(byte[].class);
kyroClasses.add(byte[][].class);
+ kyroClasses.add(String[].class);
+ kyroClasses.add(String[][].class);
kyroClasses.add(org.apache.spark.sql.types.Decimal.class);
kyroClasses.add(scala.math.BigDecimal.class);
kyroClasses.add(java.math.BigDecimal.class);
@@ -553,6 +578,61 @@ public class SparkCubing extends AbstractApplication {
kyroClasses.add(java.math.RoundingMode.class);
kyroClasses.add(java.util.ArrayList.class);
kyroClasses.add(java.util.LinkedList.class);
+ kyroClasses.add(java.util.HashSet.class);
+ kyroClasses.add(java.util.LinkedHashSet.class);
+ kyroClasses.add(java.util.LinkedHashMap.class);
+ kyroClasses.add(java.util.TreeMap.class);
+ kyroClasses.add(java.util.concurrent.ConcurrentHashMap.class);
+
+ kyroClasses.add(java.util.HashMap.class);
+ kyroClasses.add(java.util.Properties.class);
+ kyroClasses.add(org.apache.kylin.metadata.model.ColumnDesc[].class);
+ kyroClasses.add(org.apache.kylin.metadata.model.JoinTableDesc[].class);
+ kyroClasses.add(org.apache.kylin.metadata.model.TblColRef[].class);
+ kyroClasses.add(org.apache.kylin.metadata.model.DataModelDesc.RealizationCapacity.class);
+ kyroClasses.add(org.apache.kylin.metadata.model.DataModelDesc.TableKind.class);
+ kyroClasses.add(org.apache.kylin.metadata.model.PartitionDesc.DefaultPartitionConditionBuilder.class);
+ kyroClasses.add(org.apache.kylin.metadata.model.PartitionDesc.PartitionType.class);
+ kyroClasses.add(org.apache.kylin.cube.model.CubeDesc.DeriveInfo.class);
+ kyroClasses.add(org.apache.kylin.cube.model.CubeDesc.DeriveType.class);
+ kyroClasses.add(org.apache.kylin.cube.model.HBaseColumnFamilyDesc[].class);
+ kyroClasses.add(org.apache.kylin.cube.model.HBaseColumnDesc[].class);
+ kyroClasses.add(org.apache.kylin.metadata.model.MeasureDesc[].class);
+ kyroClasses.add(org.apache.kylin.cube.model.RowKeyColDesc[].class);
+ kyroClasses.add(org.apache.kylin.common.util.Array.class);
+ kyroClasses.add(org.apache.kylin.metadata.model.Segments.class);
+ kyroClasses.add(org.apache.kylin.metadata.realization.RealizationStatusEnum.class);
+ kyroClasses.add(org.apache.kylin.metadata.model.SegmentStatusEnum.class);
+ kyroClasses.add(org.apache.kylin.measure.BufferedMeasureCodec.class);
+ kyroClasses.add(org.apache.kylin.cube.kv.RowKeyColumnIO.class);
+ kyroClasses.add(org.apache.kylin.measure.MeasureCodec.class);
+ kyroClasses.add(org.apache.kylin.measure.MeasureAggregator[].class);
+ kyroClasses.add(org.apache.kylin.metadata.datatype.DataTypeSerializer[].class);
+ kyroClasses.add(org.apache.kylin.cube.kv.CubeDimEncMap.class);
+ kyroClasses.add(org.apache.kylin.measure.basic.BasicMeasureType.class);
+ kyroClasses.add(org.apache.kylin.common.util.SplittedBytes[].class);
+ kyroClasses.add(org.apache.kylin.common.util.SplittedBytes.class);
+ kyroClasses.add(org.apache.kylin.cube.kv.RowKeyEncoderProvider.class);
+ kyroClasses.add(org.apache.kylin.cube.kv.RowKeyEncoder.class);
+ kyroClasses.add(org.apache.kylin.measure.basic.BigDecimalIngester.class);
+ kyroClasses.add(org.apache.kylin.dimension.DictionaryDimEnc.class);
+ kyroClasses.add(org.apache.kylin.dimension.IntDimEnc.class);
+ kyroClasses.add(org.apache.kylin.dimension.BooleanDimEnc.class);
+ kyroClasses.add(org.apache.kylin.dimension.DateDimEnc.class);
+ kyroClasses.add(org.apache.kylin.dimension.FixedLenDimEnc.class);
+ kyroClasses.add(org.apache.kylin.dimension.FixedLenHexDimEnc.class);
+ kyroClasses.add(org.apache.kylin.dimension.IntegerDimEnc.class);
+ kyroClasses.add(org.apache.kylin.dimension.OneMoreByteVLongDimEnc.class);
+ kyroClasses.add(org.apache.kylin.dimension.TimeDimEnc.class);
+ kyroClasses.add(org.apache.kylin.cube.model.AggregationGroup.HierarchyMask.class);
+ kyroClasses.add(org.apache.kylin.measure.topn.DoubleDeltaSerializer.class);
+ kyroClasses.add(org.apache.kylin.measure.topn.Counter.class);
+
+ try {
+ kyroClasses.add(Class.forName("com.google.common.collect.EmptyImmutableList"));
+ } catch (ClassNotFoundException e) {
+ logger.error("failed to load class", e);
+ }
ArrayList<String> result = Lists.newArrayList();
for (Class kyroClass : kyroClasses) {
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
----------------------------------------------------------------------
diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
new file mode 100644
index 0000000..53c1f96
--- /dev/null
+++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkCubingByLayer.java
@@ -0,0 +1,416 @@
+/*
+ * 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.engine.spark;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import org.apache.commons.cli.Option;
+import org.apache.commons.cli.OptionBuilder;
+import org.apache.commons.cli.Options;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
+import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.AbstractApplication;
+import org.apache.kylin.common.util.ByteArray;
+import org.apache.kylin.common.util.ClassUtil;
+import org.apache.kylin.common.util.OptionsHelper;
+import org.apache.kylin.common.util.Pair;
+import org.apache.kylin.cube.CubeInstance;
+import org.apache.kylin.cube.CubeManager;
+import org.apache.kylin.cube.CubeSegment;
+import org.apache.kylin.cube.common.RowKeySplitter;
+import org.apache.kylin.cube.cuboid.Cuboid;
+import org.apache.kylin.cube.cuboid.CuboidScheduler;
+import org.apache.kylin.cube.kv.AbstractRowKeyEncoder;
+import org.apache.kylin.cube.kv.RowKeyEncoderProvider;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.CubeJoinedFlatTableEnrich;
+import org.apache.kylin.engine.EngineFactory;
+import org.apache.kylin.engine.mr.BatchCubingJobBuilder2;
+import org.apache.kylin.engine.mr.HadoopUtil;
+import org.apache.kylin.engine.mr.common.BaseCuboidBuilder;
+import org.apache.kylin.engine.mr.common.BatchConstants;
+import org.apache.kylin.engine.mr.common.CubeStatsReader;
+import org.apache.kylin.engine.mr.common.NDCuboidBuilder;
+import org.apache.kylin.measure.BufferedMeasureCodec;
+import org.apache.kylin.measure.MeasureAggregators;
+import org.apache.kylin.measure.MeasureIngester;
+import org.apache.kylin.metadata.model.MeasureDesc;
+import org.apache.spark.SparkConf;
+import org.apache.spark.SparkFiles;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.api.java.function.PairFlatMapFunction;
+import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.DataFrame;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.hive.HiveContext;
+import org.apache.spark.storage.StorageLevel;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Tuple2;
+
+import javax.annotation.Nullable;
+import java.io.File;
+import java.io.FileFilter;
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.kylin.engine.spark.SparkCubing.getKyroClasses;
+
+/**
+ */
+public class SparkCubingByLayer extends AbstractApplication implements Serializable {
+
+ protected static final Logger logger = LoggerFactory.getLogger(SparkCubingByLayer.class);
+
+ public static final Option OPTION_INPUT_PATH = OptionBuilder.withArgName("path").hasArg().isRequired(true).withDescription("Hive Intermediate Table").create("hiveTable");
+ public static final Option OPTION_CUBE_NAME = OptionBuilder.withArgName(BatchConstants.ARG_CUBE_NAME).hasArg().isRequired(true).withDescription("Cube Name").create(BatchConstants.ARG_CUBE_NAME);
+ public static final Option OPTION_SEGMENT_ID = OptionBuilder.withArgName("segment").hasArg().isRequired(true).withDescription("Cube Segment Id").create("segmentId");
+ public static final Option OPTION_CONF_PATH = OptionBuilder.withArgName("confPath").hasArg().isRequired(true).withDescription("Configuration Path").create("confPath");
+ public static final Option OPTION_OUTPUT_PATH = OptionBuilder.withArgName(BatchConstants.ARG_OUTPUT).hasArg().isRequired(true).withDescription("Cube output path").create(BatchConstants.ARG_OUTPUT);
+
+ private Options options;
+
+ public SparkCubingByLayer() {
+ options = new Options();
+ options.addOption(OPTION_INPUT_PATH);
+ options.addOption(OPTION_CUBE_NAME);
+ options.addOption(OPTION_SEGMENT_ID);
+ options.addOption(OPTION_CONF_PATH);
+ options.addOption(OPTION_OUTPUT_PATH);
+ }
+
+ @Override
+ protected Options getOptions() {
+ return options;
+ }
+
+ private void setupClasspath(JavaSparkContext sc, String confPath) throws Exception {
+ ClassUtil.addClasspath(confPath);
+ final File[] files = new File(confPath).listFiles(new FileFilter() {
+ @Override
+ public boolean accept(File pathname) {
+ if (pathname.getAbsolutePath().endsWith(".xml")) {
+ return true;
+ }
+ if (pathname.getAbsolutePath().endsWith(".properties")) {
+ return true;
+ }
+ return false;
+ }
+ });
+ for (File file : files) {
+ sc.addFile(file.getAbsolutePath());
+ }
+ }
+
+ private static final void prepare() {
+ final File file = new File(SparkFiles.get("kylin.properties"));
+ final String confPath = file.getParentFile().getAbsolutePath();
+ logger.info("conf directory:" + confPath);
+ System.setProperty(KylinConfig.KYLIN_CONF, confPath);
+ ClassUtil.addClasspath(confPath);
+ }
+
+ @Override
+ protected void execute(OptionsHelper optionsHelper) throws Exception {
+ final String hiveTable = optionsHelper.getOptionValue(OPTION_INPUT_PATH);
+ final String cubeName = optionsHelper.getOptionValue(OPTION_CUBE_NAME);
+ final String segmentId = optionsHelper.getOptionValue(OPTION_SEGMENT_ID);
+ final String confPath = optionsHelper.getOptionValue(OPTION_CONF_PATH);
+ final String outputPath = optionsHelper.getOptionValue(OPTION_OUTPUT_PATH);
+
+ SparkConf conf = new SparkConf().setAppName("Cubing Application");
+ //serialization conf
+ conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
+ conf.set("spark.kryo.registrationRequired", "true");
+ final Iterable<String> allClasses = Iterables.filter(Iterables.concat(Lists.newArrayList(conf.get("spark.kryo.classesToRegister", "").split(",")), getKyroClasses()), new Predicate<String>() {
+ @Override
+ public boolean apply(@Nullable String input) {
+ return input != null && input.trim().length() > 0;
+ }
+ });
+ conf.set("spark.kryo.classesToRegister", StringUtils.join(allClasses, ","));
+
+ JavaSparkContext sc = new JavaSparkContext(conf);
+ setupClasspath(sc, confPath);
+ HadoopUtil.deletePath(sc.hadoopConfiguration(), new Path(outputPath));
+
+ HiveContext sqlContext = new HiveContext(sc.sc());
+ final DataFrame intermediateTable = sqlContext.table(hiveTable);
+
+ System.setProperty(KylinConfig.KYLIN_CONF, confPath);
+ final KylinConfig envConfig = KylinConfig.getInstanceFromEnv();
+ final CubeInstance cubeInstance = CubeManager.getInstance(envConfig).getCube(cubeName);
+ final CubeDesc cubeDesc = cubeInstance.getDescriptor();
+ final CubeSegment cubeSegment = cubeInstance.getSegmentById(segmentId);
+ final CubeJoinedFlatTableEnrich intermediateTableDesc = new CubeJoinedFlatTableEnrich(EngineFactory.getJoinedFlatTableDesc(cubeSegment), cubeDesc);
+
+ final KylinConfig kylinConfig = cubeDesc.getConfig();
+ final Broadcast<CubeDesc> vCubeDesc = sc.broadcast(cubeDesc);
+ final Broadcast<CubeSegment> vCubeSegment = sc.broadcast(cubeSegment);
+ final NDCuboidBuilder ndCuboidBuilder = new NDCuboidBuilder(vCubeSegment.getValue(), new RowKeyEncoderProvider(vCubeSegment.getValue()));
+
+ final Broadcast<CuboidScheduler> vCuboidScheduler = sc.broadcast(new CuboidScheduler(vCubeDesc.getValue()));
+
+ final long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
+ final Cuboid baseCuboid = Cuboid.findById(cubeDesc, baseCuboidId);
+ final int measureNum = cubeDesc.getMeasures().size();
+ final BaseCuboidBuilder baseCuboidBuilder = new BaseCuboidBuilder(kylinConfig, vCubeDesc.getValue(), vCubeSegment.getValue(), intermediateTableDesc, AbstractRowKeyEncoder.createInstance(cubeSegment, baseCuboid), MeasureIngester.create(cubeDesc.getMeasures()), cubeSegment.buildDictionaryMap());
+
+ int countMeasureIndex = 0;
+ for (MeasureDesc measureDesc : cubeDesc.getMeasures()) {
+ if (measureDesc.getFunction().isCount() == true) {
+ break;
+ } else {
+ countMeasureIndex++;
+ }
+ }
+ final CubeStatsReader cubeStatsReader = new CubeStatsReader(cubeSegment, kylinConfig);
+ boolean[] needAggr = new boolean[cubeDesc.getMeasures().size()];
+ boolean allNormalMeasure = true;
+ for (int i = 0; i < cubeDesc.getMeasures().size(); i++) {
+ needAggr[i] = !cubeDesc.getMeasures().get(i).getFunction().getMeasureType().onlyAggrInBaseCuboid();
+ allNormalMeasure = allNormalMeasure && needAggr[i];
+ }
+ logger.info("All measure are normal (agg on all cuboids) ? : " + allNormalMeasure);
+
+ StorageLevel storageLevel = StorageLevel.MEMORY_AND_DISK_SER();
+
+ // encode with dimension encoding, transform to <ByteArray, Object[]> RDD
+ final JavaPairRDD<ByteArray, Object[]> encodedBaseRDD = intermediateTable.javaRDD().mapToPair(new PairFunction<Row, ByteArray, Object[]>() {
+ transient boolean initialized = false;
+
+ @Override
+ public Tuple2<ByteArray, Object[]> call(Row row) throws Exception {
+ if (initialized == false) {
+ prepare();
+ initialized = true;
+ }
+
+ String[] rowArray = rowToArray(row);
+ baseCuboidBuilder.resetAggrs();
+ byte[] rowKey = baseCuboidBuilder.buildKey(rowArray);
+ Object[] result = baseCuboidBuilder.buildValueObjects(rowArray);
+ return new Tuple2<>(new ByteArray(rowKey), result);
+ }
+
+ private String[] rowToArray(Row row) {
+ String[] result = new String[row.size()];
+ for (int i = 0; i < row.size(); i++) {
+ final Object o = row.get(i);
+ if (o != null) {
+ result[i] = o.toString();
+ } else {
+ result[i] = null;
+ }
+ }
+ return result;
+ }
+
+ });
+
+ logger.info("encodedBaseRDD partition number: " + encodedBaseRDD.getNumPartitions());
+ Long totalCount = 0L;
+ if (kylinConfig.isSparkSanityCheckEnabled()) {
+ totalCount = encodedBaseRDD.count();
+ logger.info("encodedBaseRDD row count: " + encodedBaseRDD.count());
+ }
+
+ final MeasureAggregators measureAggregators = new MeasureAggregators(cubeDesc.getMeasures());
+ final BaseCuboidReducerFunction2 baseCuboidReducerFunction = new BaseCuboidReducerFunction2(measureNum, vCubeDesc.getValue(), measureAggregators);
+ BaseCuboidReducerFunction2 reducerFunction2 = baseCuboidReducerFunction;
+ if (allNormalMeasure == false) {
+ reducerFunction2 = new CuboidReducerFunction2(measureNum, vCubeDesc.getValue(), measureAggregators, needAggr);
+ }
+
+ final int totalLevels = cubeDesc.getBuildLevel();
+ JavaPairRDD<ByteArray, Object[]>[] allRDDs = new JavaPairRDD[totalLevels];
+ int level = 0;
+ int partition = estimateRDDPartitionNum(level, cubeStatsReader, kylinConfig);
+
+ // aggregate to calculate base cuboid
+ allRDDs[0] = encodedBaseRDD.reduceByKey(baseCuboidReducerFunction, partition).persist(storageLevel);
+ saveToHDFS(allRDDs[0], vCubeDesc.getValue(), outputPath, 0, sc.hadoopConfiguration());
+
+ // aggregate to ND cuboids
+ PairFlatMapFunction<Tuple2<ByteArray, Object[]>, ByteArray, Object[]> flatMapFunction = new CuboidFlatMap(vCubeSegment.getValue(), vCubeDesc.getValue(), vCuboidScheduler.getValue(), ndCuboidBuilder);
+
+ for (level = 1; level < totalLevels; level++) {
+ partition = estimateRDDPartitionNum(level, cubeStatsReader, kylinConfig);
+ logger.info("Level " + level + " partition number: " + partition);
+ allRDDs[level] = allRDDs[level - 1].flatMapToPair(flatMapFunction).reduceByKey(reducerFunction2, partition).persist(storageLevel);
+ if (kylinConfig.isSparkSanityCheckEnabled() == true) {
+ sanityCheck(allRDDs[level], totalCount, level, cubeStatsReader, countMeasureIndex);
+ }
+ saveToHDFS(allRDDs[level], vCubeDesc.getValue(), outputPath, level, sc.hadoopConfiguration());
+ allRDDs[level - 1].unpersist();
+ }
+ allRDDs[totalLevels - 1].unpersist();
+ logger.info("Finished on calculating all level cuboids.");
+ }
+
+ private static int estimateRDDPartitionNum(int level, CubeStatsReader statsReader, KylinConfig kylinConfig) {
+ double baseCuboidSize = statsReader.estimateLayerSize(level);
+ float rddCut = kylinConfig.getSparkRDDPartitionCutMB();
+ int partition = (int) (baseCuboidSize / rddCut);
+ partition = Math.max(kylinConfig.getSparkMinPartition(), partition);
+ partition = Math.min(kylinConfig.getSparkMaxPartition(), partition);
+ logger.debug("Estimated level " + level + " partition number: " + partition);
+ return partition;
+ }
+
+ private static void saveToHDFS(final JavaPairRDD<ByteArray, Object[]> rdd, final CubeDesc cubeDesc, final String hdfsBaseLocation, int level, Configuration conf) {
+ final String cuboidOutputPath = BatchCubingJobBuilder2.getCuboidOutputPathsByLevel(hdfsBaseLocation, level);
+ rdd.mapToPair(new PairFunction<Tuple2<ByteArray, Object[]>, org.apache.hadoop.io.Text, org.apache.hadoop.io.Text>() {
+ BufferedMeasureCodec codec = new BufferedMeasureCodec(cubeDesc.getMeasures());
+ @Override
+ public Tuple2<org.apache.hadoop.io.Text, org.apache.hadoop.io.Text> call(Tuple2<ByteArray, Object[]> tuple2) throws Exception {
+ ByteBuffer valueBuf = codec.encode(tuple2._2());
+ byte[] encodedBytes = new byte[valueBuf.position()];
+ System.arraycopy(valueBuf.array(), 0, encodedBytes, 0, valueBuf.position());
+ return new Tuple2<>(new org.apache.hadoop.io.Text(tuple2._1().array()), new org.apache.hadoop.io.Text(encodedBytes));
+ }
+ }).saveAsNewAPIHadoopFile(cuboidOutputPath, org.apache.hadoop.io.Text.class, org.apache.hadoop.io.Text.class, SequenceFileOutputFormat.class, conf);
+ logger.info("Persisting RDD for level " + level + " into " + cuboidOutputPath);
+ }
+
+ class BaseCuboidReducerFunction2 implements Function2<Object[], Object[], Object[]> {
+ CubeDesc cubeDesc;
+ int measureNum;
+ MeasureAggregators aggregators;
+
+ BaseCuboidReducerFunction2(int measureNum, CubeDesc cubeDesc, MeasureAggregators aggregators) {
+ this.cubeDesc = cubeDesc;
+ this.measureNum = measureNum;
+ this.aggregators = aggregators;
+ }
+
+ @Override
+ public Object[] call(Object[] input1, Object[] input2) throws Exception {
+ Object[] result = new Object[measureNum];
+ aggregators.aggregate(input1, input2, result);
+ return result;
+ }
+ }
+
+ class CuboidReducerFunction2 extends BaseCuboidReducerFunction2 {
+ boolean[] needAggr;
+
+ CuboidReducerFunction2(int measureNum, CubeDesc cubeDesc, MeasureAggregators aggregators, boolean[] needAggr) {
+ super(measureNum, cubeDesc, aggregators);
+ this.needAggr = needAggr;
+ }
+
+ @Override
+ public Object[] call(Object[] input1, Object[] input2) throws Exception {
+ Object[] result = new Object[measureNum];
+ aggregators.aggregate(input1, input2, result, needAggr);
+ return result;
+ }
+ }
+
+ private static final java.lang.Iterable<Tuple2<ByteArray, Object[]>> EMTPY_ITERATOR = Lists.newArrayListWithCapacity(0);
+
+ class CuboidFlatMap implements PairFlatMapFunction<Tuple2<ByteArray, Object[]>, ByteArray, Object[]> {
+
+ CubeSegment cubeSegment;
+ CubeDesc cubeDesc;
+ CuboidScheduler cuboidScheduler;
+ NDCuboidBuilder ndCuboidBuilder;
+ RowKeySplitter rowKeySplitter;
+ transient boolean initialized = false;
+
+ CuboidFlatMap(CubeSegment cubeSegment, CubeDesc cubeDesc, CuboidScheduler cuboidScheduler, NDCuboidBuilder ndCuboidBuilder) {
+ this.cubeSegment = cubeSegment;
+ this.cubeDesc = cubeDesc;
+ this.cuboidScheduler = cuboidScheduler;
+ this.ndCuboidBuilder = ndCuboidBuilder;
+ this.rowKeySplitter = new RowKeySplitter(cubeSegment, 65, 256);
+ }
+
+ @Override
+ public Iterable<Tuple2<ByteArray, Object[]>> call(Tuple2<ByteArray, Object[]> tuple2) throws Exception {
+ if (initialized == false) {
+ prepare();
+ initialized = true;
+ }
+
+ byte[] key = tuple2._1().array();
+ long cuboidId = rowKeySplitter.split(key);
+ Cuboid parentCuboid = Cuboid.findById(cubeDesc, cuboidId);
+
+ Collection<Long> myChildren = cuboidScheduler.getSpanningCuboid(cuboidId);
+
+ // if still empty or null
+ if (myChildren == null || myChildren.size() == 0) {
+ return EMTPY_ITERATOR;
+ }
+
+ List<Tuple2<ByteArray, Object[]>> tuples = Lists.newArrayListWithCapacity(myChildren.size());
+ for (Long child : myChildren) {
+ Cuboid childCuboid = Cuboid.findById(cubeDesc, child);
+ Pair<Integer, ByteArray> result = ndCuboidBuilder.buildKey(parentCuboid, childCuboid, rowKeySplitter.getSplitBuffers());
+
+ byte[] newKey = new byte[result.getFirst()];
+ System.arraycopy(result.getSecond().array(), 0, newKey, 0, result.getFirst());
+
+ tuples.add(new Tuple2<>(new ByteArray(newKey), tuple2._2()));
+ }
+
+ return tuples;
+ }
+ }
+
+ //sanity check
+
+ private void sanityCheck(JavaPairRDD<ByteArray, Object[]> rdd, Long totalCount, int thisLevel, CubeStatsReader cubeStatsReader, final int countMeasureIndex) {
+ int thisCuboidNum = cubeStatsReader.getCuboidsByLayer(thisLevel).size();
+ Long count2 = getRDDCountSum(rdd, countMeasureIndex);
+ if (count2 != totalCount * thisCuboidNum) {
+ throw new IllegalStateException(String.format("Sanity check failed, level %s, total count(*) is %s; cuboid number %s", thisLevel, count2, thisCuboidNum));
+ } else {
+ logger.info("sanity check success for level " + thisLevel + ", count(*) is " + (count2 / thisCuboidNum));
+ }
+ }
+
+ private Long getRDDCountSum(JavaPairRDD<ByteArray, Object[]> rdd, final int countMeasureIndex) {
+ final ByteArray ONE = new ByteArray();
+ Long count = rdd.mapValues(new Function<Object[], Long>() {
+ @Override
+ public Long call(Object[] objects) throws Exception {
+ return (Long) objects[countMeasureIndex]; // assume the first measure is COUNT(*)
+ }
+ }).reduce(new Function2<Tuple2<ByteArray, Long>, Tuple2<ByteArray, Long>, Tuple2<ByteArray, Long>>() {
+ @Override
+ public Tuple2<ByteArray, Long> call(Tuple2<ByteArray, Long> longTuple2, Tuple2<ByteArray, Long> longTuple22) throws Exception {
+ return new Tuple2<>(ONE, longTuple2._2() + longTuple22._2());
+ }
+ })._2();
+ return count;
+ }
+}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkExecutable.java
----------------------------------------------------------------------
diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkExecutable.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkExecutable.java
index 7c88372..644f73f 100644
--- a/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkExecutable.java
+++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/SparkExecutable.java
@@ -22,6 +22,7 @@ import java.util.Map;
import org.apache.commons.lang3.StringUtils;
import org.apache.kylin.common.KylinConfig;
+import org.apache.kylin.common.util.CliCommandExecutor;
import org.apache.kylin.common.util.Logger;
import org.apache.kylin.job.exception.ExecuteException;
import org.apache.kylin.job.execution.AbstractExecutable;
@@ -38,11 +39,16 @@ public class SparkExecutable extends AbstractExecutable {
private static final org.slf4j.Logger logger = LoggerFactory.getLogger(SparkExecutable.class);
private static final String CLASS_NAME = "className";
+ private static final String JARS = "jars";
public void setClassName(String className) {
this.setParam(CLASS_NAME, className);
}
+ public void setJars(String jars) {
+ this.setParam(JARS, jars);
+ }
+
private String formatArgs() {
StringBuilder stringBuilder = new StringBuilder();
for (Map.Entry<String, String> entry : getParams().entrySet()) {
@@ -50,6 +56,9 @@ public class SparkExecutable extends AbstractExecutable {
tmp.append("-").append(entry.getKey()).append(" ").append(entry.getValue()).append(" ");
if (entry.getKey().equals(CLASS_NAME)) {
stringBuilder.insert(0, tmp);
+ } else if (entry.getKey().equals(JARS)) {
+ // JARS is for spark-submit, not for app
+ continue;
} else {
stringBuilder.append(tmp);
}
@@ -65,12 +74,22 @@ public class SparkExecutable extends AbstractExecutable {
protected ExecuteResult doWork(ExecutableContext context) throws ExecuteException {
final KylinConfig config = context.getConfig();
Preconditions.checkNotNull(config.getSparkHome());
- Preconditions.checkNotNull(config.getSparkMaster());
+ Preconditions.checkNotNull(config.getKylinJobJarPath());
+ String sparkConf = config.getSparkConfFile();
+ String jars = this.getParam(JARS);
+
+ String jobJar = config.getKylinJobJarPath();
+
+ if (StringUtils.isEmpty(jars)) {
+ jars = jobJar;
+ }
+
try {
- String cmd = String.format("%s/bin/spark-submit --class \"org.apache.kylin.common.util.SparkEntry\" --master %s %s %s", config.getSparkHome(), config.getSparkMaster(), config.getKylinSparkJobJarPath(), formatArgs());
+ String cmd = String.format("export HADOOP_CONF_DIR=%s && %s/bin/spark-submit --class \"org.apache.kylin.common.util.SparkEntry\" --properties-file %s --jars %s %s %s", config.getSparkHadoopConfDir(), config.getSparkHome(), sparkConf, jars, jobJar, formatArgs());
logger.info("cmd:" + cmd);
final StringBuilder output = new StringBuilder();
- config.getCliCommandExecutor().execute(cmd, new Logger() {
+ CliCommandExecutor exec = new CliCommandExecutor();
+ exec.execute(cmd, new Logger() {
@Override
public void log(String message) {
output.append(message);
@@ -84,4 +103,5 @@ public class SparkExecutable extends AbstractExecutable {
return new ExecuteResult(ExecuteResult.State.ERROR, e.getLocalizedMessage());
}
}
+
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/engine-spark/src/main/java/org/apache/kylin/engine/spark/cube/DefaultTupleConverter.java
----------------------------------------------------------------------
diff --git a/engine-spark/src/main/java/org/apache/kylin/engine/spark/cube/DefaultTupleConverter.java b/engine-spark/src/main/java/org/apache/kylin/engine/spark/cube/DefaultTupleConverter.java
index 4f407ff..793cd87 100644
--- a/engine-spark/src/main/java/org/apache/kylin/engine/spark/cube/DefaultTupleConverter.java
+++ b/engine-spark/src/main/java/org/apache/kylin/engine/spark/cube/DefaultTupleConverter.java
@@ -38,7 +38,7 @@ import scala.Tuple2;
*/
public final class DefaultTupleConverter implements TupleConverter {
- private final static ThreadLocal<ByteBuffer> valueBuf = new ThreadLocal<>();
+ private final static transient ThreadLocal<ByteBuffer> valueBuf = new ThreadLocal<>();
private final CubeSegment segment;
private final int measureCount;
private final Map<TblColRef, Integer> columnLengthMap;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/examples/test_case_data/sandbox/kylin-spark-conf.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin-spark-conf.properties b/examples/test_case_data/sandbox/kylin-spark-conf.properties
new file mode 100644
index 0000000..b4a0c97
--- /dev/null
+++ b/examples/test_case_data/sandbox/kylin-spark-conf.properties
@@ -0,0 +1,28 @@
+spark.yarn.submit.file.replication=1
+spark.yarn.executor.memoryOverhead=200
+spark.yarn.driver.memoryOverhead=384
+#spark.master=local[4]
+#spark.submit.deployMode=client
+spark.master=yarn
+spark.submit.deployMode=cluster
+spark.eventLog.enabled=true
+spark.yarn.scheduler.heartbeat.interval-ms=5000
+spark.yarn.preserve.staging.files=true
+spark.yarn.queue=default
+spark.yarn.containerLauncherMaxThreads=25
+spark.yarn.max.executor.failures=3
+spark.eventLog.dir=hdfs\:///spark-history
+spark.history.kerberos.enabled=true
+spark.history.provider=org.apache.spark.deploy.history.FsHistoryProvider
+spark.history.ui.port=18080
+spark.history.fs.logDirectory=hdfs\:///spark-history
+spark.executor.memory=1G
+spark.storage.memoryFraction=0.3
+spark.executor.cores=1
+spark.executor.instances=1
+spark.history.kerberos.keytab=none
+spark.history.kerberos.principal=none
+spark.yarn.jar=hdfs://sandbox.hortonworks.com:8020/apps/spark/spark-assembly-1.6.3-hadoop2.6.0.jar
+spark.driver.extraJavaOptions=-Dhdp.version=current
+spark.yarn.am.extraJavaOptions=-Dhdp.version=current
+spark.executor.extraJavaOptions=-Dhdp.version=current
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/examples/test_case_data/sandbox/kylin.properties
----------------------------------------------------------------------
diff --git a/examples/test_case_data/sandbox/kylin.properties b/examples/test_case_data/sandbox/kylin.properties
index 1724619..a011911 100644
--- a/examples/test_case_data/sandbox/kylin.properties
+++ b/examples/test_case_data/sandbox/kylin.properties
@@ -79,6 +79,8 @@ kylin.job.remote-cli-working-dir=/tmp/kylin
# Max count of concurrent jobs running
kylin.job.max-concurrent-jobs=10
+kylin.source.hive.redistribute-flat-table=false
+
# Time interval to check hadoop job status
kylin.engine.mr.yarn-check-interval-seconds=10
@@ -154,3 +156,11 @@ kylin.server.query-metrics-percentiles-intervals=60, 360, 3600
# Env DEV|QA|PROD
kylin.env=DEV
+kylin.source.hive.keep-flat-table=true
+
+### Spark as Engine ###
+#kylin.engine.spark.env.hadoop-conf-dir=/etc/hadoop/conf
+kylin.engine.spark.env.hadoop-conf-dir=../examples/test_case_data/sandbox
+kylin.engine.spark.spark-home=/usr/local/spark
+kylin.engine.spark.properties-file=../examples/test_case_data/sandbox/kylin-spark-conf.properties
+kylin.engine.spark.sanity-check-enabled=false
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0b323cf..0c9a8cb 100644
--- a/pom.xml
+++ b/pom.xml
@@ -84,7 +84,7 @@
<commons-math3.version>3.6.1</commons-math3.version>
<!-- Spark -->
- <spark.version>1.3.0</spark.version>
+ <spark.version>1.6.3</spark.version>
<!-- Utility -->
<log4j.version>1.2.17</log4j.version>
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/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 a9c8321..f905fe1 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
@@ -327,10 +327,10 @@ public class CubeController extends BasicController {
throw new InternalErrorException("Cannot find cube '" + cubeName + "'");
}
- if (cube.getSegments() != null && cube.getBuildingSegments().size() > 0) {
- int num = cube.getBuildingSegments().size();
- throw new InternalErrorException("Cannot purge cube '" + cubeName + "' as there is " + num + " building " + (num > 1 ? "segment(s)." : "segment. Discard the related job first."));
- }
+// if (cube.getSegments() != null && cube.getBuildingSegments().size() > 0) {
+// int num = cube.getBuildingSegments().size();
+// throw new InternalErrorException("Cannot purge cube '" + cubeName + "' as there is " + num + " building " + (num > 1 ? "segment(s)." : "segment. Discard the related job first."));
+// }
return cubeService.purgeCube(cube);
} catch (Exception e) {
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/server/pom.xml
----------------------------------------------------------------------
diff --git a/server/pom.xml b/server/pom.xml
index 981d64c..1d6bdc9 100644
--- a/server/pom.xml
+++ b/server/pom.xml
@@ -249,6 +249,13 @@
<scope>provided</scope>
</dependency>
+ <!-- Spark dependency -->
+ <dependency>
+ <groupId>org.apache.spark</groupId>
+ <artifactId>spark-core_2.10</artifactId>
+ <scope>provided</scope>
+ </dependency>
+
<dependency>
<groupId>org.apache.hbase</groupId>
<artifactId>hbase-testing-util</artifactId>
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
----------------------------------------------------------------------
diff --git a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
index fc2b982..763b537 100644
--- a/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
+++ b/source-hive/src/main/java/org/apache/kylin/source/hive/HiveMRInput.java
@@ -22,6 +22,7 @@ import java.io.IOException;
import java.util.Map;
import java.util.Set;
+import com.google.common.collect.Sets;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -56,8 +57,6 @@ import org.apache.kylin.metadata.model.TableDesc;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import com.google.common.collect.Sets;
-
public class HiveMRInput implements IMRInput {
public static String getTableNameForHCat(TableDesc table) {
@@ -74,7 +73,7 @@ public class HiveMRInput implements IMRInput {
public IMRTableInputFormat getTableInputFormat(TableDesc table) {
return new HiveTableInputFormat(getTableNameForHCat(table));
}
-
+
@Override
public IMRBatchMergeInputSide getBatchMergeInputSide(ISegment seg) {
return new IMRBatchMergeInputSide() {
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
index f0ad6e0..9b1a00d 100644
--- a/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
+++ b/storage-hbase/src/test/java/org/apache/kylin/storage/hbase/steps/RowValueDecoderTest.java
@@ -31,7 +31,6 @@ import org.apache.kylin.cube.model.CubeDesc;
import org.apache.kylin.cube.model.HBaseColumnDesc;
import org.apache.kylin.measure.BufferedMeasureCodec;
import org.apache.kylin.metadata.MetadataManager;
-import org.apache.kylin.metadata.datatype.LongMutable;
import org.apache.kylin.metadata.model.FunctionDesc;
import org.apache.kylin.metadata.model.MeasureDesc;
import org.junit.After;
@@ -60,8 +59,8 @@ public class RowValueDecoderTest extends LocalFileMetadataTestCase {
BigDecimal sum = new BigDecimal("333.1234567");
BigDecimal min = new BigDecimal("333.1111111");
BigDecimal max = new BigDecimal("333.1999999");
- LongMutable count = new LongMutable(2);
- LongMutable item_count = new LongMutable(100);
+ Long count = new Long(2);
+ Long item_count = new Long(100);
ByteBuffer buf = codec.encode(new Object[] { sum, min, max, count, item_count });
buf.flip();
@@ -91,7 +90,7 @@ public class RowValueDecoderTest extends LocalFileMetadataTestCase {
BigDecimal min = new BigDecimal("333.1111111");
BigDecimal max = new BigDecimal("333.1999999");
LongWritable count = new LongWritable(2);
- LongMutable item_count = new LongMutable(100);
+ Long item_count = new Long(100);
codec.encode(new Object[] { sum, min, max, count, item_count });
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/tool/pom.xml
----------------------------------------------------------------------
diff --git a/tool/pom.xml b/tool/pom.xml
index 8dba3f7..ccd546a 100644
--- a/tool/pom.xml
+++ b/tool/pom.xml
@@ -40,6 +40,10 @@
</dependency>
<dependency>
<groupId>org.apache.kylin</groupId>
+ <artifactId>kylin-engine-spark</artifactId>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.kylin</groupId>
<artifactId>kylin-source-kafka</artifactId>
</dependency>
<dependency>
[12/17] kylin git commit: KYLIN-2333 revert the changes as there is the case that “mandatory only combination is allowed”
Posted by sh...@apache.org.
KYLIN-2333 revert the changes as there is the case that \u201cmandatory only combination is allowed\u201d
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/df5faf37
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/df5faf37
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/df5faf37
Branch: refs/heads/sparkcubing-rebase
Commit: df5faf37cd3186376c28018a5a5a7e42bcea1db9
Parents: 4b5257a
Author: shaofengshi <sh...@apache.org>
Authored: Thu Jan 5 15:14:37 2017 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Jan 5 15:14:37 2017 +0800
----------------------------------------------------------------------
.../java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java | 3 +--
.../java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java | 2 +-
.../main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java | 2 +-
3 files changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/df5faf37/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
index 8fbecd3..456f615 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder.java
@@ -65,12 +65,11 @@ public class BatchCubingJobBuilder extends JobBuilderSupport {
// Phase 3: Build Cube
RowKeyDesc rowKeyDesc = seg.getCubeDesc().getRowkey();
final int groupRowkeyColumnsCount = seg.getCubeDesc().getBuildLevel();
- final int totalRowkeyColumnsCount = rowKeyDesc.getRowKeyColumns().length;
final String[] cuboidOutputTempPath = getCuboidOutputPaths(cuboidRootPath, groupRowkeyColumnsCount);
// base cuboid step
result.addTask(createBaseCuboidStep(cuboidOutputTempPath[0], jobId));
// n dim cuboid steps
- for (int i = 1; i < groupRowkeyColumnsCount; i++) {
+ for (int i = 1; i <= groupRowkeyColumnsCount; i++) {
result.addTask(createNDimensionCuboidStep(cuboidOutputTempPath[i - 1], cuboidOutputTempPath[i], i));
}
outputSide.addStepPhase3_BuildCube(result, cuboidRootPath);
http://git-wip-us.apache.org/repos/asf/kylin/blob/df5faf37/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
index 9229a75..1e3f078 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
@@ -81,7 +81,7 @@ public class BatchCubingJobBuilder2 extends JobBuilderSupport {
// base cuboid step
result.addTask(createBaseCuboidStep(cuboidOutputTempPath[0], jobId));
// n dim cuboid steps
- for (int i = 1; i < maxLevel; i++) {
+ for (int i = 1; i <= maxLevel; i++) {
result.addTask(createNDimensionCuboidStep(cuboidOutputTempPath[i - 1], cuboidOutputTempPath[i], i, jobId));
}
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/df5faf37/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
index 23fce2b..122b8ba 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
@@ -194,7 +194,7 @@ public class JobBuilderSupport {
public String[] getCuboidOutputPaths(String cuboidRootPath, int levels) {
String[] paths = new String[levels];
- for (int i = 0; i < levels; i++) {
+ for (int i = 0; i <= levels; i++) {
if (i == 0) {
paths[i] = cuboidRootPath + "base_cuboid";
} else {
[10/17] kylin git commit: KYLIN-2359 Update job build step name
Posted by sh...@apache.org.
KYLIN-2359 Update job build step name
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/9c985742
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/9c985742
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/9c985742
Branch: refs/heads/sparkcubing-rebase
Commit: 9c985742584a6bf15053d5482d13e1e01f76894d
Parents: 809b783
Author: shaofengshi <sh...@apache.org>
Authored: Thu Jan 5 12:19:42 2017 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Jan 5 12:19:42 2017 +0800
----------------------------------------------------------------------
.../org/apache/kylin/job/constant/ExecutableConstants.java | 7 ++++---
.../org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java | 4 ++--
.../org/apache/kylin/storage/hbase/steps/HBaseMRSteps.java | 4 ++--
3 files changed, 8 insertions(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/9c985742/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java b/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
index 4ae3add..11c7455 100644
--- a/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
+++ b/core-job/src/main/java/org/apache/kylin/job/constant/ExecutableConstants.java
@@ -38,9 +38,9 @@ public final class ExecutableConstants {
public static final String STEP_NAME_CREATE_FLAT_HIVE_TABLE = "Create Intermediate Flat Hive Table";
public static final String STEP_NAME_MATERIALIZE_HIVE_VIEW_IN_LOOKUP = "Materialize Hive View in Lookup Tables";
public static final String STEP_NAME_FACT_DISTINCT_COLUMNS = "Extract Fact Table Distinct Columns";
- public static final String STEP_NAME_BUILD_BASE_CUBOID = "Build Base Cuboid Data";
- public static final String STEP_NAME_BUILD_IN_MEM_CUBE = "Build Cube";
- public static final String STEP_NAME_BUILD_N_D_CUBOID = "Build N-Dimension Cuboid Data";
+ public static final String STEP_NAME_BUILD_BASE_CUBOID = "Build Base Cuboid";
+ public static final String STEP_NAME_BUILD_IN_MEM_CUBE = "Build Cube In-Mem";
+ public static final String STEP_NAME_BUILD_N_D_CUBOID = "Build N-Dimension Cuboid";
public static final String STEP_NAME_GET_CUBOID_KEY_DISTRIBUTION = "Calculate HTable Region Splits";
public static final String STEP_NAME_CREATE_HBASE_TABLE = "Create HTable";
public static final String STEP_NAME_CONVERT_CUBOID_TO_HFILE = "Convert Cuboid Data to HFile";
@@ -53,6 +53,7 @@ public final class ExecutableConstants {
public static final String STEP_NAME_HIVE_CLEANUP = "Hive Cleanup";
public static final String STEP_NAME_KAFKA_CLEANUP = "Kafka Intermediate File Cleanup";
public static final String STEP_NAME_GARBAGE_COLLECTION = "Garbage Collection";
+ public static final String STEP_NAME_GARBAGE_COLLECTION_HBASE = "Garbage Collection on HBase";
public static final String STEP_NAME_GARBAGE_COLLECTION_HDFS = "Garbage Collection on HDFS";
public static final String STEP_NAME_REDISTRIBUTE_FLAT_HIVE_TABLE = "Redistribute Flat Hive Table";
public static final String NOTIFY_EMAIL_TEMPLATE = "<div><b>Build Result of Job ${job_name}</b><pre><ul>" + "<li>Build Result: <b>${result}</b></li>" + "<li>Job Engine: ${job_engine}</li>" + "<li>Env: ${env_name}</li>" + "<li>Project: ${project_name}</li>" + "<li>Cube Name: ${cube_name}</li>" + "<li>Source Records Count: ${source_records_count}</li>" + "<li>Start Time: ${start_time}</li>" + "<li>Duration: ${duration}</li>" + "<li>MR Waiting: ${mr_waiting}</li>" + "<li>Last Update Time: ${last_update_time}</li>" + "<li>Submitter: ${submitter}</li>" + "<li>Error Log: ${error_log}</li>" + "</ul></pre><div/>";
http://git-wip-us.apache.org/repos/asf/kylin/blob/9c985742/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
index 20540a6..9229a75 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
@@ -43,13 +43,13 @@ public class BatchCubingJobBuilder2 extends JobBuilderSupport {
public BatchCubingJobBuilder2(CubeSegment newSegment, String submitter) {
super(newSegment, submitter);
this.inputSide = MRUtil.getBatchCubingInputSide(seg);
- this.outputSide = MRUtil.getBatchCubingOutputSide2((CubeSegment) seg);
+ this.outputSide = MRUtil.getBatchCubingOutputSide2(seg);
}
public CubingJob build() {
logger.info("MR_V2 new job to BUILD segment " + seg);
- final CubingJob result = CubingJob.createBuildJob((CubeSegment) seg, submitter, config);
+ final CubingJob result = CubingJob.createBuildJob(seg, submitter, config);
final String jobId = result.getId();
final String cuboidRootPath = getCuboidRootPath(jobId);
http://git-wip-us.apache.org/repos/asf/kylin/blob/9c985742/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMRSteps.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMRSteps.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMRSteps.java
index 7c2b3fd..6f69e8c 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMRSteps.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/steps/HBaseMRSteps.java
@@ -162,7 +162,7 @@ public class HBaseMRSteps extends JobBuilderSupport {
public MergeGCStep createMergeGCStep() {
MergeGCStep result = new MergeGCStep();
- result.setName(ExecutableConstants.STEP_NAME_GARBAGE_COLLECTION);
+ result.setName(ExecutableConstants.STEP_NAME_GARBAGE_COLLECTION_HBASE);
result.setOldHTables(getMergingHTables());
return result;
}
@@ -218,7 +218,7 @@ public class HBaseMRSteps extends JobBuilderSupport {
toDeletePaths.add(getFactDistinctColumnsPath(jobId));
HDFSPathGarbageCollectionStep step = new HDFSPathGarbageCollectionStep();
- step.setName(ExecutableConstants.STEP_NAME_GARBAGE_COLLECTION);
+ step.setName(ExecutableConstants.STEP_NAME_GARBAGE_COLLECTION_HBASE);
step.setDeletePaths(toDeletePaths);
step.setJobId(jobId);
[09/17] kylin git commit: KYLIN-2342 When NoClassDefFoundError
occurred in building cube, no error in kylin.log
Posted by sh...@apache.org.
KYLIN-2342 When NoClassDefFoundError occurred in building cube, no error in kylin.log
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/e043d5fa
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/e043d5fa
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/e043d5fa
Branch: refs/heads/sparkcubing-rebase
Commit: e043d5fad8ccf420b1c0e744ee87a237cd7babd7
Parents: e6a84c5
Author: shaofengshi <sh...@apache.org>
Authored: Thu Jan 5 10:54:33 2017 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Jan 5 12:01:36 2017 +0800
----------------------------------------------------------------------
.../org/apache/kylin/rest/controller/CubeController.java | 4 ++--
.../org/apache/kylin/rest/controller/TableController.java | 10 ++++++----
.../org/apache/kylin/rest/init/InitialTaskManager.java | 2 +-
3 files changed, 9 insertions(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/e043d5fa/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 978f477..a9c8321 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
@@ -293,7 +293,7 @@ public class CubeController extends BasicController {
}
return jobService.submitJob(cube, startTime, endTime, startOffset, endOffset, //
sourcePartitionOffsetStart, sourcePartitionOffsetEnd, CubeBuildTypeEnum.valueOf(buildType), force, submitter);
- } catch (Exception e) {
+ } catch (Throwable e) {
logger.error(e.getLocalizedMessage(), e);
throw new InternalErrorException(e.getLocalizedMessage());
}
@@ -656,7 +656,7 @@ public class CubeController extends BasicController {
cubeService.getCubeDescManager().updateCubeDesc(desc);
response.setProperty("result", "success");
response.setProperty("offsets", startOffsets.toString());
- } catch (Exception e) {
+ } catch (Throwable e) {
throw new RuntimeException(e);
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/e043d5fa/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 1ab6ba6..ac4ee2f 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
@@ -112,7 +112,7 @@ public class TableController extends BasicController {
if (request.isCalculate()) {
tableService.calculateCardinalityIfNotPresent(loaded, submitter);
}
- } catch (Exception e) {
+ } catch (Throwable e) {
logger.error("Failed to load Hive Table", e);
throw new InternalErrorException(e.getLocalizedMessage());
}
@@ -133,7 +133,7 @@ public class TableController extends BasicController {
unLoadFail.add(tableName);
}
}
- } catch (Exception e) {
+ } catch (Throwable e) {
logger.error("Failed to unload Hive Table", e);
throw new InternalErrorException(e.getLocalizedMessage());
}
@@ -175,7 +175,8 @@ public class TableController extends BasicController {
private List<String> showHiveDatabases() throws IOException {
try {
return tableService.getHiveDbNames();
- } catch (Exception e) {
+ } catch (Throwable e) {
+ logger.error(e.getLocalizedMessage(), e);
throw new InternalErrorException(e.getLocalizedMessage());
}
}
@@ -191,7 +192,8 @@ public class TableController extends BasicController {
private List<String> showHiveTables(@PathVariable String database) throws IOException {
try {
return tableService.getHiveTableNames(database);
- } catch (Exception e) {
+ } catch (Throwable e) {
+ logger.error(e.getLocalizedMessage(), e);
throw new InternalErrorException(e.getLocalizedMessage());
}
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/e043d5fa/server-base/src/main/java/org/apache/kylin/rest/init/InitialTaskManager.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/init/InitialTaskManager.java b/server-base/src/main/java/org/apache/kylin/rest/init/InitialTaskManager.java
index 5cf48cb..8bba674 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/init/InitialTaskManager.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/init/InitialTaskManager.java
@@ -52,7 +52,7 @@ public class InitialTaskManager implements InitializingBean {
InitialTask task = (InitialTask) Class.forName(taskClass).newInstance();
logger.info("Running task: " + taskClass);
task.execute();
- } catch (Exception e) {
+ } catch (Throwable e) {
logger.error("Initial task failed: " + taskClass, e);
}
}
[03/17] kylin git commit: KYLIN-2322 disable subtree cache
Posted by sh...@apache.org.
KYLIN-2322 disable subtree cache
Signed-off-by: Li Yang <li...@apache.org>
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/487f2d99
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/487f2d99
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/487f2d99
Branch: refs/heads/sparkcubing-rebase
Commit: 487f2d99dfe76566c5ff366e316bc531a694050c
Parents: e62d878
Author: xiefan46 <95...@qq.com>
Authored: Tue Jan 3 16:13:29 2017 +0800
Committer: Li Yang <li...@apache.org>
Committed: Wed Jan 4 13:43:48 2017 +0800
----------------------------------------------------------------------
.../org/apache/kylin/dict/CacheDictionary.java | 2 +-
.../apache/kylin/dict/TrieDictionaryForest.java | 31 ++++++++++++--------
2 files changed, 20 insertions(+), 13 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/487f2d99/core-dictionary/src/main/java/org/apache/kylin/dict/CacheDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/CacheDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/CacheDictionary.java
index 575358e..1b3bfa1 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/CacheDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/CacheDictionary.java
@@ -30,7 +30,7 @@ import java.util.concurrent.ConcurrentHashMap;
abstract public class CacheDictionary<T> extends Dictionary<T> {
private static final long serialVersionUID = 1L;
- transient protected boolean enableValueCache = true;
+ transient protected boolean enableValueCache = false;
transient private SoftReference<Map> valueToIdCache;
http://git-wip-us.apache.org/repos/asf/kylin/blob/487f2d99/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java
index c655854..065c3df 100755
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/TrieDictionaryForest.java
@@ -27,7 +27,6 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
-
import org.apache.kylin.common.util.ByteArray;
import org.apache.kylin.common.util.Bytes;
import org.apache.kylin.common.util.BytesUtil;
@@ -57,13 +56,17 @@ public class TrieDictionaryForest<T> extends CacheDictionary<T> {
public TrieDictionaryForest(ArrayList<TrieDictionary<T>> trees, ArrayList<ByteArray> valueDivide, //
ArrayList<Integer> accuOffset, BytesConverter<T> bytesConverter, int baseId) {
+ init(trees, valueDivide, accuOffset, bytesConverter, baseId);
+ }
+
+ private void init(ArrayList<TrieDictionary<T>> trees, ArrayList<ByteArray> valueDivide, ArrayList<Integer> accuOffset, BytesConverter<T> bytesConverter, int baseId) {
this.trees = trees;
this.valueDivide = valueDivide;
this.accuOffset = accuOffset;
this.bytesConvert = bytesConverter;
this.baseId = baseId;
initMaxValue();
- enableCache();
+ initForestCache();
}
@Override
@@ -100,7 +103,6 @@ public class TrieDictionaryForest<T> extends CacheDictionary<T> {
return maxValue;
}
-
@Override
protected int getIdFromValueBytesImpl(byte[] value, int offset, int len, int roundingFlag) throws IllegalArgumentException {
@@ -140,8 +142,6 @@ public class TrieDictionaryForest<T> extends CacheDictionary<T> {
return id;
}
-
-
@Override
protected int getValueBytesFromIdImpl(int id, byte[] returnValue, int offset) throws IllegalArgumentException {
int index = (trees.size() == 1) ? 0 : findIndexById(id);
@@ -228,19 +228,20 @@ public class TrieDictionaryForest<T> extends CacheDictionary<T> {
try {
@SuppressWarnings("unused")
int headSize = in.readInt();
- this.baseId = in.readInt();
+ int baseId = in.readInt();
String converterName = in.readUTF();
+ BytesConverter<T> bytesConverter = null;
if (converterName.isEmpty() == false)
- this.bytesConvert = ClassUtil.forName(converterName, BytesConverter.class).newInstance();
+ bytesConverter = ClassUtil.forName(converterName, BytesConverter.class).newInstance();
//init accuOffset
int accuSize = in.readInt();
- this.accuOffset = new ArrayList<>();
+ ArrayList<Integer> accuOffset = new ArrayList<>();
for (int i = 0; i < accuSize; i++) {
accuOffset.add(in.readInt());
}
//init valueDivide
int valueDivideSize = in.readInt();
- this.valueDivide = new ArrayList<>();
+ ArrayList<ByteArray> valueDivide = new ArrayList<>();
for (int i = 0; i < valueDivideSize; i++) {
int length = in.readInt();
byte[] buffer = new byte[length];
@@ -248,14 +249,13 @@ public class TrieDictionaryForest<T> extends CacheDictionary<T> {
valueDivide.add(new ByteArray(buffer, 0, buffer.length));
}
int treeSize = in.readInt();
- this.trees = new ArrayList<>();
+ ArrayList<TrieDictionary<T>> trees = new ArrayList<>();
for (int i = 0; i < treeSize; i++) {
TrieDictionary<T> dict = new TrieDictionary<>();
dict.readFields(in);
trees.add(dict);
}
- initMaxValue();
- enableCache();
+ init(trees, valueDivide, accuOffset, bytesConverter, baseId);
} catch (Exception e) {
if (e instanceof RuntimeException)
throw (RuntimeException) e;
@@ -368,4 +368,11 @@ public class TrieDictionaryForest<T> extends CacheDictionary<T> {
}
}
+ private void initForestCache() {
+ enableCache();
+ for (TrieDictionary<T> tree : trees) { //disable duplicate cache
+ tree.disableCache();
+ }
+ }
+
}
[02/17] kylin git commit: KYLIN-2317 add HybridController and
HybridService
Posted by sh...@apache.org.
KYLIN-2317 add HybridController and HybridService
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/e62d8782
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/e62d8782
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/e62d8782
Branch: refs/heads/sparkcubing-rebase
Commit: e62d87826f0b4fc5ab3b4fdbed85d2c2ffe188be
Parents: 1d6a36b
Author: Billy Liu <bi...@apache.org>
Authored: Tue Jan 3 18:28:36 2017 +0800
Committer: Billy Liu <bi...@apache.org>
Committed: Tue Jan 3 18:28:55 2017 +0800
----------------------------------------------------------------------
.../kylin/rest/controller/BasicController.java | 8 +
.../kylin/rest/controller/HybridController.java | 84 ++++++++++
.../kylin/rest/request/HybridRequest.java | 62 ++++++++
.../kylin/rest/service/HybridService.java | 155 +++++++++++++++++++
4 files changed, 309 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/e62d8782/server-base/src/main/java/org/apache/kylin/rest/controller/BasicController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/BasicController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/BasicController.java
index 0e4a70c..f61492d 100644
--- a/server-base/src/main/java/org/apache/kylin/rest/controller/BasicController.java
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/BasicController.java
@@ -20,6 +20,7 @@ package org.apache.kylin.rest.controller;
import javax.servlet.http.HttpServletRequest;
+import org.apache.commons.lang3.StringUtils;
import org.apache.kylin.rest.exception.BadRequestException;
import org.apache.kylin.rest.exception.ForbiddenException;
import org.apache.kylin.rest.exception.NotFoundException;
@@ -66,4 +67,11 @@ public class BasicController {
logger.error("", ex);
return new ErrorResponse(req.getRequestURL().toString(), ex);
}
+
+ protected void checkRequiredArg(String fieldName, Object fieldValue) {
+ if (fieldValue == null || StringUtils.isEmpty(String.valueOf(fieldValue))) {
+ throw new BadRequestException(fieldName + " is required");
+ }
+ }
+
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/e62d8782/server-base/src/main/java/org/apache/kylin/rest/controller/HybridController.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/controller/HybridController.java b/server-base/src/main/java/org/apache/kylin/rest/controller/HybridController.java
new file mode 100644
index 0000000..18156b6
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/controller/HybridController.java
@@ -0,0 +1,84 @@
+/*
+ * 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.rest.controller;
+
+import java.util.Collection;
+
+import org.apache.kylin.rest.request.HybridRequest;
+import org.apache.kylin.rest.service.HybridService;
+import org.apache.kylin.storage.hybrid.HybridInstance;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Controller;
+import org.springframework.web.bind.annotation.PathVariable;
+import org.springframework.web.bind.annotation.RequestBody;
+import org.springframework.web.bind.annotation.RequestMapping;
+import org.springframework.web.bind.annotation.RequestMethod;
+import org.springframework.web.bind.annotation.RequestParam;
+import org.springframework.web.bind.annotation.ResponseBody;
+
+@Controller
+@RequestMapping(value = "/hybrids")
+public class HybridController extends BasicController {
+
+ @Autowired
+ private HybridService hybridService;
+
+ @RequestMapping(value = "", method = RequestMethod.POST)
+ @ResponseBody
+ public HybridInstance create(@RequestBody HybridRequest request) {
+ checkRequiredArg("hybrid", request.getHybrid());
+ checkRequiredArg("project", request.getProject());
+ checkRequiredArg("model", request.getModel());
+ checkRequiredArg("cubes", request.getCubes());
+ HybridInstance instance = hybridService.createHybridCube(request.getHybrid(), request.getProject(), request.getModel(), request.getCubes());
+ return instance;
+ }
+
+ @RequestMapping(value = "", method = RequestMethod.PUT)
+ @ResponseBody
+ public HybridInstance update(@RequestBody HybridRequest request) {
+ checkRequiredArg("hybrid", request.getHybrid());
+ checkRequiredArg("project", request.getProject());
+ checkRequiredArg("model", request.getModel());
+ checkRequiredArg("cubes", request.getCubes());
+ HybridInstance instance = hybridService.updateHybridCube(request.getHybrid(), request.getProject(), request.getModel(), request.getCubes());
+ return instance;
+ }
+
+ @RequestMapping(value = "", method = RequestMethod.DELETE)
+ @ResponseBody
+ public void delete(@RequestBody HybridRequest request) {
+ checkRequiredArg("hybrid", request.getHybrid());
+ checkRequiredArg("project", request.getProject());
+ checkRequiredArg("model", request.getModel());
+ hybridService.deleteHybridCube(request.getHybrid(), request.getProject(), request.getModel());
+ }
+
+ @RequestMapping(value = "", method = RequestMethod.GET)
+ @ResponseBody
+ public Collection<HybridInstance> list(@RequestParam(required = false) String project, @RequestParam(required = false) String model) {
+ return hybridService.listHybrids(project, model);
+ }
+
+ @RequestMapping(value = "{hybrid}", method = RequestMethod.GET)
+ @ResponseBody
+ public HybridInstance get(@PathVariable String hybrid) {
+ return hybridService.getHybridInstance(hybrid);
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kylin/blob/e62d8782/server-base/src/main/java/org/apache/kylin/rest/request/HybridRequest.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/request/HybridRequest.java b/server-base/src/main/java/org/apache/kylin/rest/request/HybridRequest.java
new file mode 100644
index 0000000..68fbaa9
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/request/HybridRequest.java
@@ -0,0 +1,62 @@
+/*
+ * 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.rest.request;
+
+public class HybridRequest {
+
+ private String hybrid;
+
+ private String project;
+
+ private String model;
+
+ private String[] cubes;
+
+ public String getProject() {
+ return this.project;
+ }
+
+ public void setProject(String project) {
+ this.project = project;
+ }
+
+ public String getHybrid() {
+ return this.hybrid;
+ }
+
+ public void setHybrid(String hybrid) {
+ this.hybrid = hybrid;
+ }
+
+ public String getModel() {
+ return this.model;
+ }
+
+ public void setModel(String model) {
+ this.model = model;
+ }
+
+ public String[] getCubes() {
+ return this.cubes;
+ }
+
+ public void setCubes(String[] cubes) {
+ this.cubes = cubes;
+ }
+}
http://git-wip-us.apache.org/repos/asf/kylin/blob/e62d8782/server-base/src/main/java/org/apache/kylin/rest/service/HybridService.java
----------------------------------------------------------------------
diff --git a/server-base/src/main/java/org/apache/kylin/rest/service/HybridService.java b/server-base/src/main/java/org/apache/kylin/rest/service/HybridService.java
new file mode 100644
index 0000000..dd1636a
--- /dev/null
+++ b/server-base/src/main/java/org/apache/kylin/rest/service/HybridService.java
@@ -0,0 +1,155 @@
+/*
+ * 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.rest.service;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.metadata.model.DataModelDesc;
+import org.apache.kylin.metadata.project.ProjectInstance;
+import org.apache.kylin.metadata.project.RealizationEntry;
+import org.apache.kylin.metadata.realization.RealizationType;
+import org.apache.kylin.rest.constant.Constant;
+import org.apache.kylin.storage.hybrid.HybridInstance;
+import org.apache.kylin.tool.HybridCubeCLI;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.security.access.prepost.PostFilter;
+import org.springframework.security.access.prepost.PreAuthorize;
+import org.springframework.stereotype.Component;
+
+@Component("hybridService")
+public class HybridService extends BasicService {
+
+ private static final Logger logger = LoggerFactory.getLogger(HybridService.class);
+
+ @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or " + Constant.ACCESS_HAS_ROLE_MODELER)
+ public HybridInstance createHybridCube(String hybridName, String projectName, String modelName, String[] cubeNames) {
+ List<String> args = new ArrayList<String>();
+ args.add("-name");
+ args.add(hybridName);
+ args.add("-project");
+ args.add(projectName);
+ args.add("-model");
+ args.add(modelName);
+ args.add("-cubes");
+ args.add(StringUtils.join(cubeNames, ","));
+ args.add("-action");
+ args.add("create");
+ try {
+ HybridCubeCLI.main(args.toArray(new String[args.size()]));
+ } catch (Exception e) {
+ logger.warn("Create Hybrid Failed", e);
+ throw e;
+ }
+ return getHybridInstance(hybridName);
+ }
+
+ @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')")
+ public HybridInstance updateHybridCube(String hybridName, String projectName, String modelName, String[] cubeNames) {
+ List<String> args = new ArrayList<String>();
+ args.add("-name");
+ args.add(hybridName);
+ args.add("-project");
+ args.add(projectName);
+ args.add("-model");
+ args.add(modelName);
+ args.add("-cubes");
+ args.add(StringUtils.join(cubeNames, ","));
+ args.add("-action");
+ args.add("update");
+ try {
+ HybridCubeCLI.main(args.toArray(new String[args.size()]));
+ } catch (Exception e) {
+ logger.warn("Update Hybrid Failed", e);
+ throw e;
+ }
+ return getHybridInstance(hybridName);
+ }
+
+ @PreAuthorize(Constant.ACCESS_HAS_ROLE_ADMIN + " or hasPermission(#cube, 'ADMINISTRATION') or hasPermission(#cube, 'MANAGEMENT')")
+ public void deleteHybridCube(String hybridName, String projectName, String modelName) {
+ List<String> args = new ArrayList<String>();
+ args.add("-name");
+ args.add(hybridName);
+ args.add("-project");
+ args.add(projectName);
+ args.add("-model");
+ args.add(modelName);
+ args.add("-action");
+ args.add("delete");
+ try {
+ HybridCubeCLI.main(args.toArray(new String[args.size()]));
+ } catch (Exception e) {
+ logger.warn("Delete Hybrid Failed", e);
+ throw e;
+ }
+ }
+
+ public HybridInstance getHybridInstance(String hybridName) {
+ HybridInstance hybridInstance = getHybridManager().getHybridInstance(hybridName);
+ return hybridInstance;
+ }
+
+ @PostFilter(Constant.ACCESS_POST_FILTER_READ)
+ public List<HybridInstance> listHybrids(final String projectName, final String modelName) {
+ ProjectInstance project = (null != projectName) ? getProjectManager().getProject(projectName) : null;
+ List<HybridInstance> hybridsInProject = new ArrayList<HybridInstance>();
+
+ if (StringUtils.isEmpty(projectName)) {
+ hybridsInProject = new ArrayList(getHybridManager().listHybridInstances());
+ } else if (project == null) {
+ return new ArrayList<>();
+ } else {
+ List<RealizationEntry> realizationEntries = project.getRealizationEntries(RealizationType.HYBRID);
+ if (realizationEntries != null) {
+ for (RealizationEntry entry : realizationEntries) {
+ HybridInstance instance = getHybridManager().getHybridInstance(entry.getRealization());
+ hybridsInProject.add(instance);
+ }
+ }
+ }
+
+ DataModelDesc model = (null != modelName) ? getMetadataManager().getDataModelDesc(modelName) : null;
+ if (StringUtils.isEmpty(modelName)) {
+ return hybridsInProject;
+ } else if (model == null) {
+ return new ArrayList<>();
+ } else {
+ List<HybridInstance> hybridsInModel = new ArrayList<HybridInstance>();
+ for (HybridInstance hybridInstance : hybridsInProject) {
+ boolean hybridInModel = false;
+ for (RealizationEntry entry : hybridInstance.getRealizationEntries()) {
+ CubeDesc cubeDesc = getCubeDescManager().getCubeDesc(entry.getRealization());
+ if (cubeDesc != null && model.getName().equalsIgnoreCase(cubeDesc.getModel().getName())) {
+ hybridInModel = true;
+ break;
+ }
+ }
+ if (hybridInModel) {
+ hybridsInModel.add(hybridInstance);
+ }
+ }
+ return hybridsInModel;
+ }
+ }
+
+}
[07/17] kylin git commit: revert
53b6c8c18033f92feadd2295456fa4f31011b5bb
Posted by sh...@apache.org.
revert 53b6c8c18033f92feadd2295456fa4f31011b5bb
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/e6a84c56
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/e6a84c56
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/e6a84c56
Branch: refs/heads/sparkcubing-rebase
Commit: e6a84c56961063f4ba83919ec1f689e9f37d1719
Parents: 7e71df9
Author: Hongbin Ma <ma...@apache.org>
Authored: Thu Jan 5 11:57:29 2017 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Thu Jan 5 11:57:40 2017 +0800
----------------------------------------------------------------------
.../hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/e6a84c56/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
----------------------------------------------------------------------
diff --git a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
index de53d0d..21a0efb 100644
--- a/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
+++ b/storage-hbase/src/main/java/org/apache/kylin/storage/hbase/cube/v2/coprocessor/endpoint/CubeVisitService.java
@@ -236,8 +236,8 @@ public class CubeVisitService extends CubeVisitProtos.CubeVisitService implement
}
final MutableBoolean scanNormalComplete = new MutableBoolean(true);
- final long deadline = scanReq.getStartTime() + scanReq.getTimeout();
- logger.info("deadline is " + deadline);
+ final long deadline = serviceStartTime + scanReq.getTimeout();
+ logger.info("deadline(local) is " + deadline);
final long storagePushDownLimit = scanReq.getStoragePushDownLimit();
final CellListIterator cellListIterator = new CellListIterator() {
[14/17] kylin git commit: KYLIN-2331 by layer spark cubing
Posted by sh...@apache.org.
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregators.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregators.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregators.java
index 8951b69..710f324 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregators.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregators.java
@@ -84,6 +84,24 @@ public class MeasureAggregators implements Serializable {
}
}
+ public void aggregate(Object[] values1, Object[] values2, Object[] result) {
+ assert values1.length == values2.length && values2.length == descLength && values1.length == result.length;
+
+ for (int i = 0; i < descLength; i++) {
+ result[i] = aggs[i].aggregate(values1[i], values2[i]);
+ }
+
+ }
+
+ public void aggregate(Object[] values1, Object[] values2, Object[] result, boolean[] aggrMask) {
+ assert values1.length == values2.length && values2.length == descLength && values1.length == result.length && result.length == aggrMask.length;
+ for (int i = 0; i < descLength; i++) {
+ if (aggrMask[i]) {
+ result[i] = aggs[i].aggregate(values1[i], values2[i]);
+ }
+ }
+ }
+
public void collectStates(Object[] states) {
for (int i = 0; i < descLength; i++) {
states[i] = aggs[i].getState();
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/MeasureCodec.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureCodec.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureCodec.java
index edaf806..2d73e59 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureCodec.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureCodec.java
@@ -18,19 +18,19 @@
package org.apache.kylin.measure;
-import java.nio.ByteBuffer;
-import java.util.Collection;
-
import org.apache.kylin.metadata.datatype.DataType;
import org.apache.kylin.metadata.datatype.DataTypeSerializer;
import org.apache.kylin.metadata.model.MeasureDesc;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+
/**
* @author yangli9
*
*/
@SuppressWarnings({ "rawtypes" })
-public class MeasureCodec {
+public class MeasureCodec implements java.io.Serializable {
private int nMeasures;
private DataTypeSerializer[] serializers;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java
index 0076252..26b7298 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureIngester.java
@@ -18,14 +18,14 @@
package org.apache.kylin.measure;
-import java.util.Collection;
-import java.util.Map;
-
import org.apache.kylin.common.util.Dictionary;
import org.apache.kylin.metadata.model.MeasureDesc;
import org.apache.kylin.metadata.model.TblColRef;
-abstract public class MeasureIngester<V> {
+import java.util.Collection;
+import java.util.Map;
+
+abstract public class MeasureIngester<V> implements java.io.Serializable {
public static MeasureIngester<?> create(MeasureDesc measure) {
return measure.getFunction().getMeasureType().newIngester();
@@ -42,6 +42,10 @@ abstract public class MeasureIngester<V> {
abstract public V valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap);
+ public void reset() {
+
+ }
+
public V reEncodeDictionary(V value, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> oldDicts, Map<TblColRef, Dictionary<String>> newDicts) {
throw new UnsupportedOperationException();
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java
index 89ff382..3338c8c 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureType.java
@@ -18,11 +18,6 @@
package org.apache.kylin.measure;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
import org.apache.kylin.common.util.Dictionary;
import org.apache.kylin.metadata.model.FunctionDesc;
import org.apache.kylin.metadata.model.MeasureDesc;
@@ -32,13 +27,18 @@ import org.apache.kylin.metadata.realization.SQLDigest;
import org.apache.kylin.metadata.tuple.Tuple;
import org.apache.kylin.metadata.tuple.TupleInfo;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
/**
* MeasureType captures how a kind of aggregation is defined, how it is calculated
* during cube build, and how it is involved in query and storage scan.
*
* @param <T> the Java type of aggregation data object, e.g. HLLCounter
*/
-abstract public class MeasureType<T> {
+abstract public class MeasureType<T> implements java.io.Serializable {
/* ============================================================================
* Define
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMaxAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMaxAggregator.java
index aa42476..7a57965 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMaxAggregator.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMaxAggregator.java
@@ -43,6 +43,20 @@ public class BigDecimalMaxAggregator extends MeasureAggregator<BigDecimal> {
}
@Override
+ public BigDecimal aggregate(BigDecimal value1, BigDecimal value2) {
+ if (value1 == null) {
+ return value2;
+ } else if (value2 == null) {
+ return value1;
+ }
+
+ if (value1.compareTo(value2) > 0)
+ return value1;
+ else
+ return value2;
+ }
+
+ @Override
public BigDecimal getState() {
return max;
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMinAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMinAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMinAggregator.java
index 81193ad..71ba7fb 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMinAggregator.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalMinAggregator.java
@@ -27,24 +27,38 @@ import org.apache.kylin.measure.MeasureAggregator;
@SuppressWarnings("serial")
public class BigDecimalMinAggregator extends MeasureAggregator<BigDecimal> {
- BigDecimal max = null;
+ BigDecimal min = null;
@Override
public void reset() {
- max = null;
+ min = null;
}
@Override
public void aggregate(BigDecimal value) {
- if (max == null)
- max = value;
- else if (max.compareTo(value) > 0)
- max = value;
+ if (min == null)
+ min = value;
+ else if (min.compareTo(value) > 0)
+ min = value;
+ }
+
+ @Override
+ public BigDecimal aggregate(BigDecimal value1, BigDecimal value2) {
+ if (value1 == null) {
+ return value2;
+ } else if (value2 == null) {
+ return value1;
+ }
+
+ if (value1.compareTo(value2) > 0)
+ return value2;
+ else
+ return value1;
}
@Override
public BigDecimal getState() {
- return max;
+ return min;
}
@Override
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalSumAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalSumAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalSumAggregator.java
index 5e00c63..9f6ffc2 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalSumAggregator.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/BigDecimalSumAggregator.java
@@ -40,6 +40,16 @@ public class BigDecimalSumAggregator extends MeasureAggregator<BigDecimal> {
}
@Override
+ public BigDecimal aggregate(BigDecimal value1, BigDecimal value2) {
+ if (value1 == null) {
+ return value2;
+ } else if (value2 == null) {
+ return value1;
+ }
+ return value1.add(value2);
+ }
+
+ @Override
public BigDecimal getState() {
return sum;
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleIngester.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleIngester.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleIngester.java
index a2f3980..90ecb0d 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleIngester.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleIngester.java
@@ -22,25 +22,24 @@ import java.util.Map;
import org.apache.kylin.common.util.Dictionary;
import org.apache.kylin.measure.MeasureIngester;
-import org.apache.kylin.metadata.datatype.DoubleMutable;
import org.apache.kylin.metadata.model.MeasureDesc;
import org.apache.kylin.metadata.model.TblColRef;
-public class DoubleIngester extends MeasureIngester<DoubleMutable> {
-
- // avoid repeated object creation
- private DoubleMutable current = new DoubleMutable();
+public class DoubleIngester extends MeasureIngester<Double> {
@Override
- public DoubleMutable valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
+ public Double valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
if (values.length > 1)
throw new IllegalArgumentException();
- DoubleMutable l = current;
if (values[0] == null || values[0].length() == 0)
- l.set(0L);
+ return new Double(0);
else
- l.set(Double.parseDouble(values[0]));
- return l;
+ return Double.parseDouble(values[0]);
+ }
+
+ @Override
+ public void reset() {
+
}
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMaxAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMaxAggregator.java
index 25911e8..f33555e 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMaxAggregator.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMaxAggregator.java
@@ -19,14 +19,13 @@
package org.apache.kylin.measure.basic;
import org.apache.kylin.measure.MeasureAggregator;
-import org.apache.kylin.metadata.datatype.DoubleMutable;
/**
*/
@SuppressWarnings("serial")
-public class DoubleMaxAggregator extends MeasureAggregator<DoubleMutable> {
+public class DoubleMaxAggregator extends MeasureAggregator<Double> {
- DoubleMutable max = null;
+ Double max = null;
@Override
public void reset() {
@@ -34,15 +33,20 @@ public class DoubleMaxAggregator extends MeasureAggregator<DoubleMutable> {
}
@Override
- public void aggregate(DoubleMutable value) {
+ public void aggregate(Double value) {
if (max == null)
- max = new DoubleMutable(value.get());
- else if (max.get() < value.get())
- max.set(value.get());
+ max = value;
+ else if (max < value)
+ max = value;
}
@Override
- public DoubleMutable getState() {
+ public Double aggregate(Double value1, Double value2) {
+ return Math.max(value1, value2);
+ }
+
+ @Override
+ public Double getState() {
return max;
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMinAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMinAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMinAggregator.java
index be97deb..8e69f21 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMinAggregator.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleMinAggregator.java
@@ -19,14 +19,13 @@
package org.apache.kylin.measure.basic;
import org.apache.kylin.measure.MeasureAggregator;
-import org.apache.kylin.metadata.datatype.DoubleMutable;
/**
*/
@SuppressWarnings("serial")
-public class DoubleMinAggregator extends MeasureAggregator<DoubleMutable> {
+public class DoubleMinAggregator extends MeasureAggregator<Double> {
- DoubleMutable min = null;
+ Double min = null;
@Override
public void reset() {
@@ -34,15 +33,20 @@ public class DoubleMinAggregator extends MeasureAggregator<DoubleMutable> {
}
@Override
- public void aggregate(DoubleMutable value) {
+ public void aggregate(Double value) {
if (min == null)
- min = new DoubleMutable(value.get());
- else if (min.get() > value.get())
- min.set(value.get());
+ min = value;
+ else if (min > value)
+ min = value;
}
@Override
- public DoubleMutable getState() {
+ public Double aggregate(Double value1, Double value2) {
+ return Math.min(value1, value2);
+ }
+
+ @Override
+ public Double getState() {
return min;
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleSumAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleSumAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleSumAggregator.java
index f276817..df0ba52 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleSumAggregator.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/DoubleSumAggregator.java
@@ -19,27 +19,31 @@
package org.apache.kylin.measure.basic;
import org.apache.kylin.measure.MeasureAggregator;
-import org.apache.kylin.metadata.datatype.DoubleMutable;
/**
*/
@SuppressWarnings("serial")
-public class DoubleSumAggregator extends MeasureAggregator<DoubleMutable> {
+public class DoubleSumAggregator extends MeasureAggregator<Double> {
- DoubleMutable sum = new DoubleMutable();
+ Double sum = new Double(0);
@Override
public void reset() {
- sum.set(0.0);
+ sum = new Double(0);
}
@Override
- public void aggregate(DoubleMutable value) {
- sum.set(sum.get() + value.get());
+ public void aggregate(Double value) {
+ sum = sum + value;
}
@Override
- public DoubleMutable getState() {
+ public Double aggregate(Double value1, Double value2) {
+ return Double.valueOf(value1 + value2);
+ }
+
+ @Override
+ public Double getState() {
return sum;
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongIngester.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongIngester.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongIngester.java
index 45a1634..793acf2 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongIngester.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongIngester.java
@@ -22,25 +22,24 @@ import java.util.Map;
import org.apache.kylin.common.util.Dictionary;
import org.apache.kylin.measure.MeasureIngester;
-import org.apache.kylin.metadata.datatype.LongMutable;
import org.apache.kylin.metadata.model.MeasureDesc;
import org.apache.kylin.metadata.model.TblColRef;
-public class LongIngester extends MeasureIngester<LongMutable> {
-
- // avoid repeated object creation
- private LongMutable current = new LongMutable();
+public class LongIngester extends MeasureIngester<Long> {
@Override
- public LongMutable valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
+ public Long valueOf(String[] values, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> dictionaryMap) {
if (values.length > 1)
throw new IllegalArgumentException();
- LongMutable l = current;
if (values[0] == null || values[0].length() == 0)
- l.set(0L);
+ return new Long(0L);
else
- l.set(Long.parseLong(values[0]));
- return l;
+ return Long.valueOf(values[0]);
+ }
+
+ @Override
+ public void reset() {
+
}
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMaxAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMaxAggregator.java
index ca44f15..b9a2b94 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMaxAggregator.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMaxAggregator.java
@@ -19,14 +19,13 @@
package org.apache.kylin.measure.basic;
import org.apache.kylin.measure.MeasureAggregator;
-import org.apache.kylin.metadata.datatype.LongMutable;
/**
*/
@SuppressWarnings("serial")
-public class LongMaxAggregator extends MeasureAggregator<LongMutable> {
+public class LongMaxAggregator extends MeasureAggregator<Long> {
- LongMutable max = null;
+ Long max = null;
@Override
public void reset() {
@@ -34,15 +33,20 @@ public class LongMaxAggregator extends MeasureAggregator<LongMutable> {
}
@Override
- public void aggregate(LongMutable value) {
+ public void aggregate(Long value) {
if (max == null)
- max = new LongMutable(value.get());
- else if (max.get() < value.get())
- max.set(value.get());
+ max = value;
+ else if (max < value)
+ max = value;
}
@Override
- public LongMutable getState() {
+ public Long aggregate(Long value1, Long value2) {
+ return Math.max(value1, value2);
+ }
+
+ @Override
+ public Long getState() {
return max;
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMinAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMinAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMinAggregator.java
index dadc64e..9185142 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMinAggregator.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongMinAggregator.java
@@ -19,14 +19,13 @@
package org.apache.kylin.measure.basic;
import org.apache.kylin.measure.MeasureAggregator;
-import org.apache.kylin.metadata.datatype.LongMutable;
/**
*/
@SuppressWarnings("serial")
-public class LongMinAggregator extends MeasureAggregator<LongMutable> {
+public class LongMinAggregator extends MeasureAggregator<Long> {
- LongMutable min = null;
+ Long min = null;
@Override
public void reset() {
@@ -34,15 +33,20 @@ public class LongMinAggregator extends MeasureAggregator<LongMutable> {
}
@Override
- public void aggregate(LongMutable value) {
+ public void aggregate(Long value) {
if (min == null)
- min = new LongMutable(value.get());
- else if (min.get() > value.get())
- min.set(value.get());
+ min = value;
+ else if (min > value)
+ min = value;
}
@Override
- public LongMutable getState() {
+ public Long aggregate(Long value1, Long value2) {
+ return Math.min(value1, value2);
+ }
+
+ @Override
+ public Long getState() {
return min;
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongSumAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongSumAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongSumAggregator.java
index e7fdc9d..1f9c0d7 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongSumAggregator.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/basic/LongSumAggregator.java
@@ -19,27 +19,31 @@
package org.apache.kylin.measure.basic;
import org.apache.kylin.measure.MeasureAggregator;
-import org.apache.kylin.metadata.datatype.LongMutable;
/**
*/
@SuppressWarnings("serial")
-public class LongSumAggregator extends MeasureAggregator<LongMutable> {
+public class LongSumAggregator extends MeasureAggregator<Long> {
- LongMutable sum = new LongMutable();
+ Long sum = new Long(0L);
@Override
public void reset() {
- sum.set(0);
+ sum = new Long(0L);
}
@Override
- public void aggregate(LongMutable value) {
- sum.set(sum.get() + value.get());
+ public void aggregate(Long value) {
+ sum += value;
}
@Override
- public LongMutable getState() {
+ public Long aggregate(Long value1, Long value2) {
+ return Long.valueOf(value1 + value2);
+ }
+
+ @Override
+ public Long getState() {
return sum;
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapAggregator.java
index be72090..cd0b4bb 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapAggregator.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapAggregator.java
@@ -42,6 +42,19 @@ public class BitmapAggregator extends MeasureAggregator<BitmapCounter> {
}
@Override
+ public BitmapCounter aggregate(BitmapCounter value1, BitmapCounter value2) {
+ if (value1 == null) {
+ return new BitmapCounter(value2);
+ } else if (value2 == null) {
+ return new BitmapCounter(value1);
+ }
+
+ BitmapCounter merged = new BitmapCounter(value1);
+ merged.merge(value2);
+ return merged;
+ }
+
+ @Override
public BitmapCounter getState() {
return sum;
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
index 827390d..caab094 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapCounter.java
@@ -18,6 +18,9 @@
package org.apache.kylin.measure.bitmap;
+import org.apache.commons.io.IOUtils;
+import org.roaringbitmap.buffer.MutableRoaringBitmap;
+
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
@@ -26,13 +29,10 @@ import java.io.InputStream;
import java.nio.ByteBuffer;
import java.util.Iterator;
-import org.apache.commons.io.IOUtils;
-import org.roaringbitmap.buffer.MutableRoaringBitmap;
-
/**
* Created by sunyerui on 15/12/1.
*/
-public class BitmapCounter implements Comparable<BitmapCounter> {
+public class BitmapCounter implements Comparable<BitmapCounter>, java.io.Serializable {
private MutableRoaringBitmap bitmap = new MutableRoaringBitmap();
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
index 8e2b2f7..6ad82a1 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapMeasureType.java
@@ -138,6 +138,11 @@ public class BitmapMeasureType extends MeasureType<BitmapCounter> {
}
return retValue;
}
+
+ @Override
+ public void reset() {
+ current = new BitmapCounter();
+ }
};
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java
index 089d18c..c40f71b 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapSerializer.java
@@ -29,7 +29,7 @@ import org.apache.kylin.metadata.datatype.DataTypeSerializer;
*/
public class BitmapSerializer extends DataTypeSerializer<BitmapCounter> {
- private ThreadLocal<BitmapCounter> current = new ThreadLocal<>();
+ private transient ThreadLocal<BitmapCounter> current = new ThreadLocal<>();
public BitmapSerializer(DataType type) {
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/extendedcolumn/ExtendedColumnMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/extendedcolumn/ExtendedColumnMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/extendedcolumn/ExtendedColumnMeasureType.java
index 6fa8788..1b2cda3 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/extendedcolumn/ExtendedColumnMeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/extendedcolumn/ExtendedColumnMeasureType.java
@@ -254,6 +254,21 @@ public class ExtendedColumnMeasureType extends MeasureType<ByteArray> {
}
@Override
+ public ByteArray aggregate(ByteArray value1, ByteArray value2) {
+ if (value1 == null) {
+ return value2;
+ } else if (value2 == null) {
+ return value1;
+ } else if (!value1.equals(value2)) {
+ if (!warned) {
+ logger.warn("Extended column must be unique given same host column");
+ warned = true;
+ }
+ }
+ return value1;
+ }
+
+ @Override
public ByteArray getState() {
return byteArray;
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/hllc/DenseRegister.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/DenseRegister.java b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/DenseRegister.java
index 5b929b2..4be9f71 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/DenseRegister.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/DenseRegister.java
@@ -24,7 +24,7 @@ import java.util.Map;
/**
* Created by xiefan on 16-12-9.
*/
-public class DenseRegister implements Register {
+public class DenseRegister implements Register, java.io.Serializable {
private int m;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCAggregator.java
index 5966c04..7d5b216 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCAggregator.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCAggregator.java
@@ -46,6 +46,13 @@ public class HLLCAggregator extends MeasureAggregator<HLLCounter> {
}
@Override
+ public HLLCounter aggregate(HLLCounter value1, HLLCounter value2) {
+ HLLCounter result = new HLLCounter(value1);
+ result.merge(value2);
+ return result;
+ }
+
+ @Override
public HLLCounter getState() {
return sum;
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java
index 9601653..de36b08 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCMeasureType.java
@@ -105,6 +105,11 @@ public class HLLCMeasureType extends MeasureType<HLLCounter> {
}
return hllc;
}
+
+ @Override
+ public void reset() {
+ current = new HLLCounter(dataType.getPrecision());
+ }
};
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCSerializer.java b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCSerializer.java
index e0992c7..df0cfaf 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCSerializer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HLLCSerializer.java
@@ -18,21 +18,18 @@
package org.apache.kylin.measure.hllc;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
import org.apache.kylin.metadata.datatype.DataType;
import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
/**
* @author yangli9
*
*/
public class HLLCSerializer extends DataTypeSerializer<HLLCounter> {
- // be thread-safe and avoid repeated obj creation
- private ThreadLocal<HLLCounter> current = new ThreadLocal<HLLCounter>();
-
private int precision;
public HLLCSerializer(DataType type) {
@@ -49,7 +46,7 @@ public class HLLCSerializer extends DataTypeSerializer<HLLCounter> {
}
private HLLCounter current() {
- HLLCounter hllc = current.get();
+ HLLCounter hllc = (HLLCounter) current.get();
if (hllc == null) {
hllc = new HLLCounter(precision);
current.set(hllc);
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HyperLogLogPlusTable.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HyperLogLogPlusTable.java b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HyperLogLogPlusTable.java
index 9d8de07..5d7bfeb 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HyperLogLogPlusTable.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/HyperLogLogPlusTable.java
@@ -25,7 +25,7 @@ import java.util.TreeMap;
* @author xjiang
*
*/
-public class HyperLogLogPlusTable {
+public class HyperLogLogPlusTable implements java.io.Serializable {
// threshold and bias data taken from google's bias correction data set:
// https://docs.google.com/document/d/1gyjfMHy43U9OWBXxfaeG-3MjGzejW1dlpyMwEYAAWEI/view?fullscreen#
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/hllc/SingleValueRegister.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/SingleValueRegister.java b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/SingleValueRegister.java
index 7f612e2..ad6306a 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/SingleValueRegister.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/SingleValueRegister.java
@@ -20,7 +20,7 @@ package org.apache.kylin.measure.hllc;
/**
* Created by xiefan on 16-12-20.
*/
-public class SingleValueRegister implements Register {
+public class SingleValueRegister implements Register, java.io.Serializable {
private int singleValuePos;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/hllc/SparseRegister.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/SparseRegister.java b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/SparseRegister.java
index dd7d7c8..d47a05b 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/hllc/SparseRegister.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/hllc/SparseRegister.java
@@ -25,7 +25,7 @@ import java.util.TreeMap;
/**
* Created by xiefan on 16-12-9.
*/
-public class SparseRegister implements Register {
+public class SparseRegister implements Register, java.io.Serializable {
private Map<Integer, Byte> sparseRegister = new TreeMap<>();
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawAggregator.java
index 6f66269..c3ecc74 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawAggregator.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawAggregator.java
@@ -47,6 +47,20 @@ public class RawAggregator extends MeasureAggregator<List<ByteArray>> {
}
@Override
+ public List<ByteArray> aggregate(List<ByteArray> value1, List<ByteArray> value2) {
+ if (value1 == null) {
+ return value2;
+ } else if (value2 == null) {
+ return value1;
+ }
+
+ List<ByteArray> result = new ArrayList<>(value1.size() + value2.size());
+ result.addAll(value1);
+ result.addAll(value2);
+ return result;
+ }
+
+ @Override
public List<ByteArray> getState() {
return list;
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawSerializer.java b/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawSerializer.java
index 021c146..68a0273 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawSerializer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawSerializer.java
@@ -34,13 +34,11 @@ public class RawSerializer extends DataTypeSerializer<List<ByteArray>> {
//FIXME to config this and RowConstants.ROWVALUE_BUFFER_SIZE in properties file
public static final int RAW_BUFFER_SIZE = 1024 * 1024;//1M
- private ThreadLocal<List<ByteArray>> current = new ThreadLocal<>();
-
public RawSerializer(DataType dataType) {
}
private List<ByteArray> current() {
- List<ByteArray> l = current.get();
+ List<ByteArray> l = (List<ByteArray>) current.get();
if (l == null) {
l = new ArrayList<ByteArray>();
current.set(l);
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/topn/DoubleDeltaSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/topn/DoubleDeltaSerializer.java b/core-metadata/src/main/java/org/apache/kylin/measure/topn/DoubleDeltaSerializer.java
index 38d5b20..ac925e2 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/topn/DoubleDeltaSerializer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/topn/DoubleDeltaSerializer.java
@@ -24,7 +24,7 @@ import java.nio.ByteBuffer;
*
* http://bitcharmer.blogspot.co.uk/2013/12/how-to-serialise-array-of-doubles-with.html
*/
-public class DoubleDeltaSerializer {
+public class DoubleDeltaSerializer implements java.io.Serializable {
// first 32 bits stores meta info
static final int PRECISION_BITS = 3;
@@ -55,7 +55,6 @@ public class DoubleDeltaSerializer {
this.precision = precision;
this.multiplier = (int) Math.pow(10, precision);
- this.deltasThreadLocal = new ThreadLocal<long[]>();
}
public void serialize(double[] values, ByteBuffer buf) {
@@ -112,6 +111,10 @@ public class DoubleDeltaSerializer {
int len = values.length - 1;
len = Math.max(0, len);
+ if (deltasThreadLocal == null) {
+ deltasThreadLocal = new ThreadLocal<>();
+ }
+
long[] deltas = deltasThreadLocal.get();
if (deltas == null || deltas.length < len) {
deltas = new long[len];
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNAggregator.java
index ef997eb..b5e316f 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNAggregator.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNAggregator.java
@@ -45,6 +45,15 @@ public class TopNAggregator extends MeasureAggregator<TopNCounter<ByteArray>> {
}
@Override
+ public TopNCounter<ByteArray> aggregate(TopNCounter<ByteArray> value1, TopNCounter<ByteArray> value2) {
+ TopNCounter<ByteArray> aggregated = new TopNCounter<>(capacity * 2);
+ aggregated.merge(value1);
+ aggregated.merge(value2);
+ aggregated.retain(capacity);
+ return aggregated;
+ }
+
+ @Override
public TopNCounter<ByteArray> getState() {
sum.retain(capacity);
return sum;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNCounter.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNCounter.java b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNCounter.java
index caf7961..5e4b91e 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNCounter.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNCounter.java
@@ -40,7 +40,7 @@ import com.google.common.collect.Maps;
*
* @param <T> type of data in the stream to be summarized
*/
-public class TopNCounter<T> implements Iterable<Counter<T>> {
+public class TopNCounter<T> implements Iterable<Counter<T>>, java.io.Serializable {
public static final int EXTRA_SPACE_RATE = 50;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureType.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureType.java b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureType.java
index c29af6c..8c8b5a6 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureType.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/topn/TopNMeasureType.java
@@ -24,6 +24,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
+import com.google.common.collect.Lists;
import org.apache.commons.lang3.StringUtils;
import org.apache.kylin.common.util.ByteArray;
import org.apache.kylin.common.util.Dictionary;
@@ -46,8 +47,6 @@ import org.apache.kylin.metadata.tuple.TupleInfo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import com.google.common.collect.Lists;
-
public class TopNMeasureType extends MeasureType<TopNCounter<ByteArray>> {
private static final Logger logger = LoggerFactory.getLogger(TopNMeasureType.class);
@@ -156,6 +155,7 @@ public class TopNMeasureType extends MeasureType<TopNCounter<ByteArray>> {
return topNCounter;
}
+
@Override
public TopNCounter<ByteArray> reEncodeDictionary(TopNCounter<ByteArray> value, MeasureDesc measureDesc, Map<TblColRef, Dictionary<String>> oldDicts, Map<TblColRef, Dictionary<String>> newDicts) {
TopNCounter<ByteArray> topNCounter = value;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BigDecimalSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BigDecimalSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BigDecimalSerializer.java
index 64968b8..b5043f5 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BigDecimalSerializer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/BigDecimalSerializer.java
@@ -35,7 +35,7 @@ public class BigDecimalSerializer extends DataTypeSerializer<BigDecimal> {
private static final Logger logger = LoggerFactory.getLogger(BigDecimalSerializer.class);
final DataType type;
- transient final int maxLength;
+ final int maxLength;
transient int avoidVerbose = 0;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/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 acb6de1..3d485d2 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
@@ -18,40 +18,26 @@
package org.apache.kylin.metadata.datatype;
-import java.nio.ByteBuffer;
-
import org.apache.commons.lang.ArrayUtils;
import org.apache.commons.lang.BooleanUtils;
-public class BooleanSerializer extends DataTypeSerializer<LongMutable> {
+import java.nio.ByteBuffer;
- public final static String[] TRUE_VALUE_SET = { "true", "t", "on", "yes" };
+public class BooleanSerializer extends DataTypeSerializer<Long> {
- // be thread-safe and avoid repeated obj creation
- private ThreadLocal<LongMutable> current = new ThreadLocal<LongMutable>();
+ public final static String[] TRUE_VALUE_SET = { "true", "t", "on", "yes" };
public BooleanSerializer(DataType type) {
}
@Override
- public void serialize(LongMutable value, ByteBuffer out) {
- out.putLong(value.get());
- }
-
- private LongMutable current() {
- LongMutable l = current.get();
- if (l == null) {
- l = new LongMutable();
- current.set(l);
- }
- return l;
+ public void serialize(Long value, ByteBuffer out) {
+ out.putLong(value);
}
@Override
- public LongMutable deserialize(ByteBuffer in) {
- LongMutable l = current();
- l.set(in.getLong());
- return l;
+ public Long deserialize(ByteBuffer in) {
+ return in.getLong();
}
@Override
@@ -70,12 +56,10 @@ public class BooleanSerializer extends DataTypeSerializer<LongMutable> {
}
@Override
- public LongMutable valueOf(String str) {
- LongMutable l = current();
+ public Long valueOf(String str) {
if (str == null)
- l.set(0L);
+ return Long.valueOf(0L);
else
- l.set(BooleanUtils.toInteger(ArrayUtils.contains(TRUE_VALUE_SET, str.toLowerCase())));
- return l;
+ return Long.valueOf(BooleanUtils.toInteger(ArrayUtils.contains(TRUE_VALUE_SET, str.toLowerCase())));
}
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataTypeSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataTypeSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataTypeSerializer.java
index a739377..a4a35a4 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataTypeSerializer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DataTypeSerializer.java
@@ -18,19 +18,21 @@
package org.apache.kylin.metadata.datatype;
-import java.nio.ByteBuffer;
-import java.util.Map;
-
+import com.google.common.collect.Maps;
import org.apache.kylin.common.util.BytesSerializer;
-import com.google.common.collect.Maps;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.nio.ByteBuffer;
+import java.util.Map;
/**
* Note: the implementations MUST be thread-safe.
*/
-abstract public class DataTypeSerializer<T> implements BytesSerializer<T> {
+abstract public class DataTypeSerializer<T> implements BytesSerializer<T>, java.io.Serializable {
final static Map<String, Class<?>> implementations = Maps.newHashMap();
+ protected transient ThreadLocal current = new ThreadLocal();
static {
implementations.put("char", StringSerializer.class);
implementations.put("varchar", StringSerializer.class);
@@ -94,4 +96,9 @@ abstract public class DataTypeSerializer<T> implements BytesSerializer<T> {
else
return value.toString();
}
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ current = new ThreadLocal();
+ }
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DateTimeSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DateTimeSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DateTimeSerializer.java
index 07f98b3..5101766 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DateTimeSerializer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DateTimeSerializer.java
@@ -18,37 +18,23 @@
package org.apache.kylin.metadata.datatype;
-import java.nio.ByteBuffer;
-
import org.apache.kylin.common.util.DateFormat;
-public class DateTimeSerializer extends DataTypeSerializer<LongMutable> {
+import java.nio.ByteBuffer;
- // be thread-safe and avoid repeated obj creation
- private ThreadLocal<LongMutable> current = new ThreadLocal<LongMutable>();
+public class DateTimeSerializer extends DataTypeSerializer<Long> {
public DateTimeSerializer(DataType type) {
}
@Override
- public void serialize(LongMutable value, ByteBuffer out) {
- out.putLong(value.get());
- }
-
- private LongMutable current() {
- LongMutable l = current.get();
- if (l == null) {
- l = new LongMutable();
- current.set(l);
- }
- return l;
+ public void serialize(Long value, ByteBuffer out) {
+ out.putLong(value);
}
@Override
- public LongMutable deserialize(ByteBuffer in) {
- LongMutable l = current();
- l.set(in.getLong());
- return l;
+ public Long deserialize(ByteBuffer in) {
+ return in.getLong();
}
@Override
@@ -67,8 +53,8 @@ public class DateTimeSerializer extends DataTypeSerializer<LongMutable> {
}
@Override
- public LongMutable valueOf(String str) {
- return new LongMutable(DateFormat.stringToMillis(str));
+ public Long valueOf(String str) {
+ return Long.valueOf(DateFormat.stringToMillis(str));
}
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DoubleSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DoubleSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DoubleSerializer.java
index 976dc51..20cfff3 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DoubleSerializer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/DoubleSerializer.java
@@ -22,33 +22,19 @@ import java.nio.ByteBuffer;
/**
*/
-public class DoubleSerializer extends DataTypeSerializer<DoubleMutable> {
-
- // be thread-safe and avoid repeated obj creation
- private ThreadLocal<DoubleMutable> current = new ThreadLocal<DoubleMutable>();
+public class DoubleSerializer extends DataTypeSerializer<Double> {
public DoubleSerializer(DataType type) {
}
@Override
- public void serialize(DoubleMutable value, ByteBuffer out) {
- out.putDouble(value.get());
- }
-
- private DoubleMutable current() {
- DoubleMutable d = current.get();
- if (d == null) {
- d = new DoubleMutable();
- current.set(d);
- }
- return d;
+ public void serialize(Double value, ByteBuffer out) {
+ out.putDouble(value);
}
@Override
- public DoubleMutable deserialize(ByteBuffer in) {
- DoubleMutable d = current();
- d.set(in.getDouble());
- return d;
+ public Double deserialize(ByteBuffer in) {
+ return in.getDouble();
}
@Override
@@ -67,7 +53,7 @@ public class DoubleSerializer extends DataTypeSerializer<DoubleMutable> {
}
@Override
- public DoubleMutable valueOf(String str) {
- return new DoubleMutable(Double.parseDouble(str));
+ public Double valueOf(String str) {
+ return Double.parseDouble(str);
}
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/Int4Serializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/Int4Serializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/Int4Serializer.java
index 7b95505..0e82e11 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/Int4Serializer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/Int4Serializer.java
@@ -18,39 +18,25 @@
package org.apache.kylin.metadata.datatype;
-import java.nio.ByteBuffer;
-
import org.apache.kylin.common.util.BytesUtil;
+import java.nio.ByteBuffer;
+
/**
*/
-public class Int4Serializer extends DataTypeSerializer<IntMutable> {
-
- // be thread-safe and avoid repeated obj creation
- private ThreadLocal<IntMutable> current = new ThreadLocal<IntMutable>();
+public class Int4Serializer extends DataTypeSerializer<Integer> {
public Int4Serializer(DataType type) {
}
@Override
- public void serialize(IntMutable value, ByteBuffer out) {
- BytesUtil.writeUnsigned(value.get(), 4, out);
- }
-
- private IntMutable current() {
- IntMutable l = current.get();
- if (l == null) {
- l = new IntMutable();
- current.set(l);
- }
- return l;
+ public void serialize(Integer value, ByteBuffer out) {
+ BytesUtil.writeUnsigned(value, 4, out);
}
@Override
- public IntMutable deserialize(ByteBuffer in) {
- IntMutable l = current();
- l.set(BytesUtil.readUnsigned(in, 4));
- return l;
+ public Integer deserialize(ByteBuffer in) {
+ return BytesUtil.readUnsigned(in, 4);
}
@Override
@@ -69,7 +55,7 @@ public class Int4Serializer extends DataTypeSerializer<IntMutable> {
}
@Override
- public IntMutable valueOf(String str) {
- return new IntMutable(Integer.parseInt(str));
+ public Integer valueOf(String str) {
+ return Integer.parseInt(str);
}
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/Long8Serializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/Long8Serializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/Long8Serializer.java
index fa333b2..7dd5aa7 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/Long8Serializer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/Long8Serializer.java
@@ -18,39 +18,26 @@
package org.apache.kylin.metadata.datatype;
-import java.nio.ByteBuffer;
-
import org.apache.kylin.common.util.BytesUtil;
+import java.nio.ByteBuffer;
+
/**
*/
-public class Long8Serializer extends DataTypeSerializer<LongMutable> {
-
- // be thread-safe and avoid repeated obj creation
- private ThreadLocal<LongMutable> current = new ThreadLocal<LongMutable>();
+public class Long8Serializer extends DataTypeSerializer<Long> {
public Long8Serializer(DataType type) {
}
@Override
- public void serialize(LongMutable value, ByteBuffer out) {
- BytesUtil.writeLong(value.get(), out);
+ public void serialize(Long value, ByteBuffer out) {
+ BytesUtil.writeLong(value, out);
}
- private LongMutable current() {
- LongMutable l = current.get();
- if (l == null) {
- l = new LongMutable();
- current.set(l);
- }
- return l;
- }
@Override
- public LongMutable deserialize(ByteBuffer in) {
- LongMutable l = current();
- l.set(BytesUtil.readLong(in));
- return l;
+ public Long deserialize(ByteBuffer in) {
+ return BytesUtil.readLong(in);
}
@Override
@@ -69,7 +56,7 @@ public class Long8Serializer extends DataTypeSerializer<LongMutable> {
}
@Override
- public LongMutable valueOf(String str) {
- return new LongMutable(Long.parseLong(str));
+ public Long valueOf(String str) {
+ return Long.parseLong(str);
}
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/LongSerializer.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/LongSerializer.java b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/LongSerializer.java
index 9306a70..605dcd7 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/LongSerializer.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/datatype/LongSerializer.java
@@ -18,39 +18,25 @@
package org.apache.kylin.metadata.datatype;
-import java.nio.ByteBuffer;
-
import org.apache.kylin.common.util.BytesUtil;
+import java.nio.ByteBuffer;
+
/**
*/
-public class LongSerializer extends DataTypeSerializer<LongMutable> {
-
- // be thread-safe and avoid repeated obj creation
- private ThreadLocal<LongMutable> current = new ThreadLocal<LongMutable>();
+public class LongSerializer extends DataTypeSerializer<Long> {
public LongSerializer(DataType type) {
}
@Override
- public void serialize(LongMutable value, ByteBuffer out) {
- BytesUtil.writeVLong(value.get(), out);
- }
-
- private LongMutable current() {
- LongMutable l = current.get();
- if (l == null) {
- l = new LongMutable();
- current.set(l);
- }
- return l;
+ public void serialize(Long value, ByteBuffer out) {
+ BytesUtil.writeVLong(value, out);
}
@Override
- public LongMutable deserialize(ByteBuffer in) {
- LongMutable l = current();
- l.set(BytesUtil.readVLong(in));
- return l;
+ public Long deserialize(ByteBuffer in) {
+ return BytesUtil.readVLong(in);
}
@Override
@@ -75,7 +61,7 @@ public class LongSerializer extends DataTypeSerializer<LongMutable> {
}
@Override
- public LongMutable valueOf(String str) {
- return new LongMutable(Long.parseLong(str));
+ public Long valueOf(String str) {
+ return Long.parseLong(str);
}
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
index 4d89e1a..ce5888d 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/FunctionDesc.java
@@ -18,27 +18,27 @@
package org.apache.kylin.metadata.model;
-import java.util.ArrayList;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.kylin.measure.MeasureType;
-import org.apache.kylin.measure.MeasureTypeFactory;
-import org.apache.kylin.measure.basic.BasicMeasureType;
-import org.apache.kylin.metadata.datatype.DataType;
-
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.google.common.collect.Lists;
import com.google.common.collect.Sets;
+import org.apache.kylin.measure.MeasureType;
+import org.apache.kylin.measure.MeasureTypeFactory;
+import org.apache.kylin.measure.basic.BasicMeasureType;
+import org.apache.kylin.metadata.datatype.DataType;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
/**
*/
@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class FunctionDesc {
+public class FunctionDesc implements Serializable {
public static FunctionDesc newInstance(String expression, ParameterDesc param, String returnType) {
FunctionDesc r = new FunctionDesc();
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinDesc.java
index 6489244..dd1500b 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinDesc.java
@@ -18,17 +18,18 @@
package org.apache.kylin.metadata.model;
-import java.util.Arrays;
-
import com.fasterxml.jackson.annotation.JsonAutoDetect;
import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
+import java.io.Serializable;
+import java.util.Arrays;
+
/**
*/
@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class JoinDesc {
+public class JoinDesc implements Serializable {
// inner, left, right, outer...
@JsonProperty("type")
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinTableDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinTableDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinTableDesc.java
index 5d0409a..51e5787 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinTableDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinTableDesc.java
@@ -25,8 +25,10 @@ import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
import com.fasterxml.jackson.annotation.JsonProperty;
+import java.io.Serializable;
+
@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class JoinTableDesc {
+public class JoinTableDesc implements Serializable {
@JsonProperty("table")
private String table;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinsTree.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinsTree.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinsTree.java
index a0b267d..c132d0e 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinsTree.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinsTree.java
@@ -18,15 +18,16 @@
package org.apache.kylin.metadata.model;
+import com.google.common.base.Preconditions;
+
+import java.io.Serializable;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
-import com.google.common.base.Preconditions;
-
-public class JoinsTree {
+public class JoinsTree implements Serializable {
final Map<String, Chain> tableChains = new LinkedHashMap<>();
@@ -111,7 +112,7 @@ public class JoinsTree {
return chain.join;
}
- static class Chain {
+ static class Chain implements java.io.Serializable {
TableRef table; // pk side
JoinDesc join;
Chain fkSide;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/metadata/model/MeasureDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/MeasureDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/MeasureDesc.java
index 253b06b..c0719d2 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/MeasureDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/MeasureDesc.java
@@ -18,18 +18,19 @@
package org.apache.kylin.metadata.model;
-import java.util.Objects;
-
import com.fasterxml.jackson.annotation.JsonAutoDetect;
-import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonProperty;
+import java.io.Serializable;
+import java.util.Objects;
+
/**
*/
@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class MeasureDesc {
+public class MeasureDesc implements Serializable {
@JsonProperty("name")
private String name;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/metadata/model/ModelDimensionDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ModelDimensionDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ModelDimensionDesc.java
index 6460f71..3c5c5f1 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ModelDimensionDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ModelDimensionDesc.java
@@ -18,17 +18,17 @@
package org.apache.kylin.metadata.model;
-import java.util.List;
-
-import org.apache.kylin.common.util.StringUtil;
-
import com.fasterxml.jackson.annotation.JsonAutoDetect;
import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kylin.common.util.StringUtil;
+
+import java.io.Serializable;
+import java.util.List;
/**
*/
@JsonAutoDetect(fieldVisibility = JsonAutoDetect.Visibility.NONE, getterVisibility = JsonAutoDetect.Visibility.NONE, isGetterVisibility = JsonAutoDetect.Visibility.NONE, setterVisibility = JsonAutoDetect.Visibility.NONE)
-public class ModelDimensionDesc {
+public class ModelDimensionDesc implements Serializable {
@JsonProperty("table")
private String table;
@JsonProperty("columns")
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/metadata/model/ParameterDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ParameterDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ParameterDesc.java
index c14d061..8ad20a8 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/ParameterDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/ParameterDesc.java
@@ -18,21 +18,22 @@
package org.apache.kylin.metadata.model;
-import java.io.UnsupportedEncodingException;
-import java.util.Arrays;
-import java.util.List;
-
import com.fasterxml.jackson.annotation.JsonAutoDetect;
-import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
+import java.io.Serializable;
+import java.io.UnsupportedEncodingException;
+import java.util.Arrays;
+import java.util.List;
+
/**
*/
@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class ParameterDesc {
+public class ParameterDesc implements Serializable {
public static ParameterDesc newInstance(Object... objs) {
if (objs.length == 0)
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java
index 9925990..c6e6425 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/PartitionDesc.java
@@ -27,10 +27,12 @@ import com.fasterxml.jackson.annotation.JsonAutoDetect;
import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
import com.fasterxml.jackson.annotation.JsonProperty;
+import java.io.Serializable;
+
/**
*/
@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class PartitionDesc {
+public class PartitionDesc implements Serializable {
public static enum PartitionType {
APPEND, //
@@ -175,7 +177,7 @@ public class PartitionDesc {
String buildDateRangeCondition(PartitionDesc partDesc, long startInclusive, long endExclusive);
}
- public static class DefaultPartitionConditionBuilder implements IPartitionConditionBuilder {
+ public static class DefaultPartitionConditionBuilder implements IPartitionConditionBuilder, java.io.Serializable {
@Override
public String buildDateRangeCondition(PartitionDesc partDesc, long startInclusive, long endExclusive) {
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableRef.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableRef.java b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableRef.java
index 7089eba..0d9b442 100644
--- a/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableRef.java
+++ b/core-metadata/src/main/java/org/apache/kylin/metadata/model/TableRef.java
@@ -18,25 +18,28 @@
package org.apache.kylin.metadata.model;
+import com.google.common.collect.Maps;
+
+import java.io.Serializable;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
-import com.google.common.collect.Maps;
+public class TableRef implements Serializable {
-public class TableRef {
-
- final private DataModelDesc model;
+ final transient private DataModelDesc model;
final private String alias;
final private TableDesc table;
final private Map<String, TblColRef> columns;
+ final private String modelName;
TableRef(DataModelDesc model, String alias, TableDesc table) {
this.model = model;
+ this.modelName = model.getName();
this.alias = alias;
this.table = table;
this.columns = Maps.newLinkedHashMap();
-
+
for (ColumnDesc col : table.getColumns()) {
columns.put(col.getName(), new TblColRef(this, col));
}
@@ -94,7 +97,7 @@ public class TableRef {
TableRef t = (TableRef) o;
- if ((model == null ? t.model == null : model.getName().equals(t.model.getName())) == false)
+ if ((modelName == null ? t.modelName != null : modelName.equals(t.modelName)) == false)
return false;
if ((alias == null ? t.alias == null : alias.equals(t.alias)) == false)
return false;
@@ -107,7 +110,7 @@ public class TableRef {
@Override
public int hashCode() {
int result = 0;
- result = 31 * result + model.getName().hashCode();
+ result = 31 * result + modelName.hashCode();
result = 31 * result + alias.hashCode();
result = 31 * result + table.getIdentity().hashCode();
return result;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/test/java/org/apache/kylin/measure/AggregatorMemEstimateTest.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/test/java/org/apache/kylin/measure/AggregatorMemEstimateTest.java b/core-metadata/src/test/java/org/apache/kylin/measure/AggregatorMemEstimateTest.java
index 40e5b29..3d48ac2 100644
--- a/core-metadata/src/test/java/org/apache/kylin/measure/AggregatorMemEstimateTest.java
+++ b/core-metadata/src/test/java/org/apache/kylin/measure/AggregatorMemEstimateTest.java
@@ -38,8 +38,6 @@ import org.apache.kylin.measure.extendedcolumn.ExtendedColumnMeasureType;
import org.apache.kylin.measure.hllc.HLLCAggregator;
import org.apache.kylin.measure.hllc.HLLCounter;
import org.apache.kylin.metadata.datatype.DataType;
-import org.apache.kylin.metadata.datatype.DoubleMutable;
-import org.apache.kylin.metadata.datatype.LongMutable;
import org.github.jamm.MemoryMeter;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@@ -61,7 +59,7 @@ public class AggregatorMemEstimateTest extends LocalFileMetadataTestCase {
}
private List<? extends MeasureAggregator> basicAggregators() {
- LongMutable longVal = new LongMutable(1000);
+ Long longVal = new Long(1000);
LongMinAggregator longMin = new LongMinAggregator();
LongMaxAggregator longMax = new LongMaxAggregator();
LongSumAggregator longSum = new LongSumAggregator();
@@ -69,7 +67,7 @@ public class AggregatorMemEstimateTest extends LocalFileMetadataTestCase {
longMax.aggregate(longVal);
longSum.aggregate(longVal);
- DoubleMutable doubleVal = new DoubleMutable(1.0);
+ Double doubleVal = new Double(1.0);
DoubleMinAggregator doubleMin = new DoubleMinAggregator();
DoubleMaxAggregator doubleMax = new DoubleMaxAggregator();
DoubleSumAggregator doubleSum = new DoubleSumAggregator();
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
----------------------------------------------------------------------
diff --git a/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java b/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
index dde0481..fcd0182 100644
--- a/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
+++ b/core-storage/src/test/java/org/apache/kylin/storage/gtrecord/DictGridTableTest.java
@@ -52,7 +52,6 @@ import org.apache.kylin.gridtable.GridTable;
import org.apache.kylin.gridtable.IGTScanner;
import org.apache.kylin.gridtable.memstore.GTSimpleMemStore;
import org.apache.kylin.metadata.datatype.DataType;
-import org.apache.kylin.metadata.datatype.LongMutable;
import org.apache.kylin.metadata.filter.ColumnTupleFilter;
import org.apache.kylin.metadata.filter.CompareTupleFilter;
import org.apache.kylin.metadata.filter.ConstantTupleFilter;
@@ -499,16 +498,16 @@ public class DictGridTableTest extends LocalFileMetadataTestCase {
GTRecord r = new GTRecord(table.getInfo());
GTBuilder builder = table.rebuild();
- builder.write(r.setValues("2015-01-14", "30", "Yang", new LongMutable(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-14", "30", "Luke", new LongMutable(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-15", "20", "Dong", new LongMutable(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-15", "20", "Jason", new LongMutable(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-15", "30", "Xu", new LongMutable(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-16", "20", "Mahone", new LongMutable(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-16", "20", "Qianhao", new LongMutable(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-16", "30", "George", new LongMutable(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-16", "30", "Shaofeng", new LongMutable(10), new BigDecimal("10.5")));
- builder.write(r.setValues("2015-01-17", "10", "Kejia", new LongMutable(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-14", "30", "Yang", new Long(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-14", "30", "Luke", new Long(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-15", "20", "Dong", new Long(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-15", "20", "Jason", new Long(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-15", "30", "Xu", new Long(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-16", "20", "Mahone", new Long(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-16", "20", "Qianhao", new Long(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-16", "30", "George", new Long(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-16", "30", "Shaofeng", new Long(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-17", "10", "Kejia", new Long(10), new BigDecimal("10.5")));
builder.close();
return table;
@@ -524,34 +523,34 @@ public class DictGridTableTest extends LocalFileMetadataTestCase {
for (int i = 0; i < 100000; i++) {
for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-14", "30", "Yang", new LongMutable(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-14", "30", "Yang", new Long(10), new BigDecimal("10.5")));
for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-14", "30", "Luke", new LongMutable(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-14", "30", "Luke", new Long(10), new BigDecimal("10.5")));
for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-15", "20", "Dong", new LongMutable(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-15", "20", "Dong", new Long(10), new BigDecimal("10.5")));
for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-15", "20", "Jason", new LongMutable(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-15", "20", "Jason", new Long(10), new BigDecimal("10.5")));
for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-15", "30", "Xu", new LongMutable(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-15", "30", "Xu", new Long(10), new BigDecimal("10.5")));
for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-16", "20", "Mahone", new LongMutable(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-16", "20", "Mahone", new Long(10), new BigDecimal("10.5")));
for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-16", "20", "Qianhao", new LongMutable(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-16", "20", "Qianhao", new Long(10), new BigDecimal("10.5")));
for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-16", "30", "George", new LongMutable(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-16", "30", "George", new Long(10), new BigDecimal("10.5")));
for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-16", "30", "Shaofeng", new LongMutable(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-16", "30", "Shaofeng", new Long(10), new BigDecimal("10.5")));
for (int j = 0; j < 10; j++)
- builder.write(r.setValues("2015-01-17", "10", "Kejia", new LongMutable(10), new BigDecimal("10.5")));
+ builder.write(r.setValues("2015-01-17", "10", "Kejia", new Long(10), new BigDecimal("10.5")));
}
builder.close();
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
index 1e3f078..700f821 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/BatchCubingJobBuilder2.java
@@ -31,6 +31,7 @@ import org.apache.kylin.engine.mr.steps.NDCuboidJob;
import org.apache.kylin.engine.mr.steps.SaveStatisticsStep;
import org.apache.kylin.job.constant.ExecutableConstants;
import org.apache.kylin.job.engine.JobEngineConfig;
+import org.apache.kylin.job.execution.AbstractExecutable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -96,7 +97,7 @@ public class BatchCubingJobBuilder2 extends JobBuilderSupport {
return result;
}
- private MapReduceExecutable createInMemCubingStep(String jobId, String cuboidRootPath) {
+ protected AbstractExecutable createInMemCubingStep(String jobId, String cuboidRootPath) {
// base cuboid job
MapReduceExecutable cubeStep = new MapReduceExecutable();
@@ -113,7 +114,7 @@ public class BatchCubingJobBuilder2 extends JobBuilderSupport {
cubeStep.setMapReduceParams(cmd.toString());
cubeStep.setMapReduceJobClass(getInMemCuboidJob());
- cubeStep.setCounterSaveAs(CubingJob.SOURCE_RECORD_COUNT + "," + CubingJob.SOURCE_SIZE_BYTES + "," + CubingJob.CUBE_SIZE_BYTES);
+// cubeStep.setCounterSaveAs(CubingJob.SOURCE_RECORD_COUNT + "," + CubingJob.SOURCE_SIZE_BYTES + "," + CubingJob.CUBE_SIZE_BYTES);
return cubeStep;
}
@@ -140,7 +141,7 @@ public class BatchCubingJobBuilder2 extends JobBuilderSupport {
baseCuboidStep.setMapReduceParams(cmd.toString());
baseCuboidStep.setMapReduceJobClass(getBaseCuboidJob());
- baseCuboidStep.setCounterSaveAs(CubingJob.SOURCE_RECORD_COUNT + "," + CubingJob.SOURCE_SIZE_BYTES);
+// baseCuboidStep.setCounterSaveAs(CubingJob.SOURCE_RECORD_COUNT + "," + CubingJob.SOURCE_SIZE_BYTES);
return baseCuboidStep;
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/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 88692a0..60bf5d9 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
@@ -36,7 +36,7 @@ import org.slf4j.LoggerFactory;
public class HadoopUtil {
@SuppressWarnings("unused")
private static final Logger logger = LoggerFactory.getLogger(HadoopUtil.class);
- private static final ThreadLocal<Configuration> hadoopConfig = new ThreadLocal<>();
+ private static final transient ThreadLocal<Configuration> hadoopConfig = new ThreadLocal<>();
public static void setCurrentConfiguration(Configuration conf) {
hadoopConfig.set(conf);
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
index 122b8ba..73aec95 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/JobBuilderSupport.java
@@ -75,6 +75,7 @@ public class JobBuilderSupport {
appendExecCmdParameters(cmd, BatchConstants.ARG_JOB_NAME, "Kylin_Fact_Distinct_Columns_" + seg.getRealization().getName() + "_Step");
appendExecCmdParameters(cmd, BatchConstants.ARG_CUBING_JOB_ID, jobId);
result.setMapReduceParams(cmd.toString());
+ result.setCounterSaveAs(CubingJob.SOURCE_RECORD_COUNT + "," + CubingJob.SOURCE_SIZE_BYTES);
return result;
}
@@ -204,4 +205,13 @@ public class JobBuilderSupport {
return paths;
}
+ public static String getCuboidOutputPathsByLevel(String cuboidRootPath, int level) {
+ if (level == 0) {
+ return cuboidRootPath + "base_cuboid";
+ } else {
+ return cuboidRootPath + level + "level_cuboid";
+ }
+ }
+
+
}
[04/17] kylin git commit: minor,
remove unused dependencies from query module
Posted by sh...@apache.org.
minor, remove unused dependencies from query module
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/5be4bbab
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/5be4bbab
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/5be4bbab
Branch: refs/heads/sparkcubing-rebase
Commit: 5be4bbab639049b596d77f1a16fc676fda484e03
Parents: 487f2d9
Author: lidongsjtu <li...@apache.org>
Authored: Wed Jan 4 14:08:39 2017 +0800
Committer: lidongsjtu <li...@apache.org>
Committed: Wed Jan 4 14:25:02 2017 +0800
----------------------------------------------------------------------
query/pom.xml | 100 ++---------------------------------------------------
1 file changed, 2 insertions(+), 98 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/5be4bbab/query/pom.xml
----------------------------------------------------------------------
diff --git a/query/pom.xml b/query/pom.xml
index 51f9370..b7cbf6d 100644
--- a/query/pom.xml
+++ b/query/pom.xml
@@ -20,7 +20,6 @@
<project xmlns="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">
<modelVersion>4.0.0</modelVersion>
- <!-- The basics. -->
<artifactId>kylin-query</artifactId>
<packaging>jar</packaging>
<name>Apache Kylin - Query</name>
@@ -30,13 +29,8 @@
<groupId>org.apache.kylin</groupId>
<artifactId>kylin</artifactId>
<version>2.0.0-SNAPSHOT</version>
-
</parent>
- <properties>
- </properties>
-
- <!-- Dependencies. -->
<dependencies>
<dependency>
<groupId>org.apache.kylin</groupId>
@@ -63,99 +57,9 @@
<scope>test</scope>
</dependency>
<dependency>
- <groupId>org.apache.kylin</groupId>
- <artifactId>kylin-storage-hbase</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.kylin</groupId>
- <artifactId>kylin-storage-hbase</artifactId>
- <type>test-jar</type>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>xerces</groupId>
- <artifactId>xercesImpl</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>xalan</groupId>
- <artifactId>xalan</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>com.h2database</groupId>
- <artifactId>h2</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.dbunit</groupId>
- <artifactId>dbunit</artifactId>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-common</artifactId>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-mapreduce-client-core</artifactId>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-hdfs</artifactId>
- <scope>provided</scope>
- <!-- protobuf version conflict with hbase -->
- <exclusions>
- <exclusion>
- <groupId>com.google.protobuf</groupId>
- <artifactId>protobuf-java</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
- <groupId>org.apache.hbase</groupId>
- <artifactId>hbase-client</artifactId>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hbase</groupId>
- <artifactId>hbase-server</artifactId>
- <scope>provided</scope>
- <!-- version conflict with hadoop2.2 -->
- <exclusions>
- <exclusion>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
- <groupId>org.apache.hadoop</groupId>
- <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hive</groupId>
- <artifactId>hive-jdbc</artifactId>
- <scope>provided</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.hbase</groupId>
- <artifactId>hbase-testing-util</artifactId>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
<scope>test</scope>
- <exclusions>
- <exclusion>
- <groupId>javax.servlet</groupId>
- <artifactId>servlet-api</artifactId>
- </exclusion>
- <exclusion>
- <groupId>javax.servlet.jsp</groupId>
- <artifactId>jsp-api</artifactId>
- </exclusion>
- </exclusions>
</dependency>
</dependencies>
</project>
[15/17] kylin git commit: KYLIN-2331 by layer spark cubing
Posted by sh...@apache.org.
KYLIN-2331 by layer spark cubing
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/a69a3b61
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/a69a3b61
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/a69a3b61
Branch: refs/heads/sparkcubing-rebase
Commit: a69a3b61462959c12e7020e9a11b7d850b5120da
Parents: df5faf3
Author: shaofengshi <sh...@apache.org>
Authored: Sun Dec 25 15:59:16 2016 +0800
Committer: shaofengshi <sh...@apache.org>
Committed: Thu Jan 5 15:20:48 2017 +0800
----------------------------------------------------------------------
assembly/pom.xml | 5 +-
.../main/config/assemblies/source-assembly.xml | 3 -
.../java/org/apache/kylin/job/DeployUtil.java | 1 -
build/conf/kylin-spark-conf.properties | 26 ++
build/conf/kylin.properties | 20 +
.../apache/kylin/common/KylinConfigBase.java | 64 ++-
.../kylin/common/persistence/ResourceStore.java | 35 ++
.../org/apache/kylin/common/util/Array.java | 2 +-
.../apache/kylin/common/util/SplittedBytes.java | 2 +-
.../java/org/apache/kylin/cube/CubeSegment.java | 2 +-
.../kylin/cube/common/RowKeySplitter.java | 15 +-
.../org/apache/kylin/cube/cuboid/Cuboid.java | 27 +-
.../kylin/cube/cuboid/CuboidScheduler.java | 15 +-
.../cube/inmemcubing/InMemCubeBuilder.java | 2 +
.../kylin/cube/kv/AbstractRowKeyEncoder.java | 6 +-
.../org/apache/kylin/cube/kv/CubeDimEncMap.java | 7 +-
.../apache/kylin/cube/kv/RowKeyColumnIO.java | 2 +-
.../org/apache/kylin/cube/kv/RowKeyEncoder.java | 19 +-
.../kylin/cube/kv/RowKeyEncoderProvider.java | 2 +-
.../kylin/cube/model/AggregationGroup.java | 4 +-
.../org/apache/kylin/cube/model/CubeDesc.java | 2 +-
.../cube/model/CubeJoinedFlatTableDesc.java | 2 +-
.../cube/model/CubeJoinedFlatTableEnrich.java | 6 +-
.../apache/kylin/cube/model/DictionaryDesc.java | 2 +-
.../apache/kylin/cube/model/DimensionDesc.java | 13 +-
.../kylin/cube/model/HBaseColumnDesc.java | 11 +-
.../kylin/cube/model/HBaseColumnFamilyDesc.java | 9 +-
.../kylin/cube/model/HBaseMappingDesc.java | 15 +-
.../apache/kylin/cube/model/HierarchyDesc.java | 2 +-
.../apache/kylin/cube/model/RowKeyColDesc.java | 2 +-
.../org/apache/kylin/cube/model/RowKeyDesc.java | 17 +-
.../org/apache/kylin/cube/model/SelectRule.java | 2 +-
.../apache/kylin/gridtable/UnitTestSupport.java | 41 +-
.../gridtable/AggregationCacheMemSizeTest.java | 6 +-
.../gridtable/AggregationCacheSpillTest.java | 5 +-
.../kylin/gridtable/SimpleGridTableTest.java | 11 +-
.../gridtable/SimpleInvertedIndexTest.java | 3 +-
.../metadata/measure/MeasureCodecTest.java | 6 +-
.../apache/kylin/dict/AppendTrieDictionary.java | 6 +-
.../org/apache/kylin/dict/NumberDictionary.java | 2 +-
.../apache/kylin/dict/NumberDictionary2.java | 2 +-
.../dict/NumberDictionaryForestBuilder.java | 4 +-
.../apache/kylin/dict/StringBytesConverter.java | 2 +-
.../kylin/job/execution/ExecutableManager.java | 21 +-
.../kylin/dimension/AbstractDateDimEnc.java | 12 +-
.../apache/kylin/dimension/BooleanDimEnc.java | 4 +-
.../kylin/dimension/DictionaryDimEnc.java | 21 +-
.../apache/kylin/dimension/FixedLenDimEnc.java | 4 +-
.../kylin/dimension/FixedLenHexDimEnc.java | 4 +-
.../org/apache/kylin/dimension/IntDimEnc.java | 4 +-
.../apache/kylin/dimension/IntegerDimEnc.java | 4 +-
.../kylin/dimension/OneMoreByteVLongDimEnc.java | 14 +-
.../kylin/measure/BufferedMeasureCodec.java | 10 +-
.../apache/kylin/measure/MeasureAggregator.java | 2 +
.../kylin/measure/MeasureAggregators.java | 18 +
.../org/apache/kylin/measure/MeasureCodec.java | 8 +-
.../apache/kylin/measure/MeasureIngester.java | 12 +-
.../org/apache/kylin/measure/MeasureType.java | 12 +-
.../measure/basic/BigDecimalMaxAggregator.java | 14 +
.../measure/basic/BigDecimalMinAggregator.java | 28 +-
.../measure/basic/BigDecimalSumAggregator.java | 10 +
.../kylin/measure/basic/DoubleIngester.java | 19 +-
.../measure/basic/DoubleMaxAggregator.java | 20 +-
.../measure/basic/DoubleMinAggregator.java | 20 +-
.../measure/basic/DoubleSumAggregator.java | 18 +-
.../kylin/measure/basic/LongIngester.java | 19 +-
.../kylin/measure/basic/LongMaxAggregator.java | 20 +-
.../kylin/measure/basic/LongMinAggregator.java | 20 +-
.../kylin/measure/basic/LongSumAggregator.java | 18 +-
.../kylin/measure/bitmap/BitmapAggregator.java | 13 +
.../kylin/measure/bitmap/BitmapCounter.java | 8 +-
.../kylin/measure/bitmap/BitmapMeasureType.java | 5 +
.../kylin/measure/bitmap/BitmapSerializer.java | 2 +-
.../ExtendedColumnMeasureType.java | 15 +
.../kylin/measure/hllc/DenseRegister.java | 2 +-
.../kylin/measure/hllc/HLLCAggregator.java | 7 +
.../kylin/measure/hllc/HLLCMeasureType.java | 5 +
.../kylin/measure/hllc/HLLCSerializer.java | 11 +-
.../measure/hllc/HyperLogLogPlusTable.java | 2 +-
.../kylin/measure/hllc/SingleValueRegister.java | 2 +-
.../kylin/measure/hllc/SparseRegister.java | 2 +-
.../apache/kylin/measure/raw/RawAggregator.java | 14 +
.../apache/kylin/measure/raw/RawSerializer.java | 4 +-
.../measure/topn/DoubleDeltaSerializer.java | 7 +-
.../kylin/measure/topn/TopNAggregator.java | 9 +
.../apache/kylin/measure/topn/TopNCounter.java | 2 +-
.../kylin/measure/topn/TopNMeasureType.java | 4 +-
.../metadata/datatype/BigDecimalSerializer.java | 2 +-
.../metadata/datatype/BooleanSerializer.java | 36 +-
.../metadata/datatype/DataTypeSerializer.java | 17 +-
.../metadata/datatype/DateTimeSerializer.java | 30 +-
.../metadata/datatype/DoubleSerializer.java | 28 +-
.../kylin/metadata/datatype/Int4Serializer.java | 32 +-
.../metadata/datatype/Long8Serializer.java | 31 +-
.../kylin/metadata/datatype/LongSerializer.java | 32 +-
.../kylin/metadata/model/FunctionDesc.java | 22 +-
.../apache/kylin/metadata/model/JoinDesc.java | 7 +-
.../kylin/metadata/model/JoinTableDesc.java | 4 +-
.../apache/kylin/metadata/model/JoinsTree.java | 9 +-
.../kylin/metadata/model/MeasureDesc.java | 9 +-
.../metadata/model/ModelDimensionDesc.java | 10 +-
.../kylin/metadata/model/ParameterDesc.java | 13 +-
.../kylin/metadata/model/PartitionDesc.java | 6 +-
.../apache/kylin/metadata/model/TableRef.java | 17 +-
.../measure/AggregatorMemEstimateTest.java | 6 +-
.../storage/gtrecord/DictGridTableTest.java | 41 +-
.../kylin/engine/mr/BatchCubingJobBuilder2.java | 7 +-
.../org/apache/kylin/engine/mr/HadoopUtil.java | 2 +-
.../kylin/engine/mr/JobBuilderSupport.java | 10 +
.../engine/mr/common/BaseCuboidBuilder.java | 173 ++++++++
.../kylin/engine/mr/common/CubeStatsReader.java | 5 +
.../kylin/engine/mr/common/NDCuboidBuilder.java | 96 +++++
.../engine/mr/steps/BaseCuboidMapperBase.java | 131 +-----
.../kylin/engine/mr/steps/NDCuboidMapper.java | 58 +--
.../kylin/engine/mr/steps/CubeReducerTest.java | 3 +-
engine-spark/pom.xml | 9 +
.../engine/spark/SparkBatchCubingEngine2.java | 33 ++
.../spark/SparkBatchCubingJobBuilder2.java | 85 ++++
.../apache/kylin/engine/spark/SparkCubing.java | 88 +++-
.../kylin/engine/spark/SparkCubingByLayer.java | 416 +++++++++++++++++++
.../kylin/engine/spark/SparkExecutable.java | 26 +-
.../spark/cube/DefaultTupleConverter.java | 2 +-
.../sandbox/kylin-spark-conf.properties | 28 ++
.../test_case_data/sandbox/kylin.properties | 10 +
pom.xml | 2 +-
.../kylin/rest/controller/CubeController.java | 8 +-
server/pom.xml | 7 +
.../apache/kylin/source/hive/HiveMRInput.java | 5 +-
.../hbase/steps/RowValueDecoderTest.java | 7 +-
tool/pom.xml | 4 +
130 files changed, 1736 insertions(+), 731 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/assembly/pom.xml
----------------------------------------------------------------------
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 65e3a8c..873abff 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -47,6 +47,10 @@
<groupId>org.apache.kylin</groupId>
<artifactId>kylin-engine-mr</artifactId>
</dependency>
+ <dependency>
+ <groupId>org.apache.kylin</groupId>
+ <artifactId>kylin-engine-spark</artifactId>
+ </dependency>
<!-- Env & Test -->
<dependency>
@@ -173,7 +177,6 @@
<shadedClassifierName>job</shadedClassifierName>
<artifactSet>
<excludes>
- <exclude>io.netty:*</exclude>
<exclude>org.apache.zookeeper:*</exclude>
<exclude>net.sf.ehcache:*</exclude>
<exclude>org.apache.httpcomponents:*</exclude>
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/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
index fad45aa..92584b5 100644
--- a/assembly/src/main/config/assemblies/source-assembly.xml
+++ b/assembly/src/main/config/assemblies/source-assembly.xml
@@ -97,9 +97,6 @@ limitations under the License.
<exclude>%regex[(?!((?!${project.build.directory}/)[^/]+/)*src/)(.*/)?docs(/.*)?]
</exclude>
- <!-- exclude unmaintained -->
- <exclude>%regex[(?!((?!${project.build.directory}/)))?engine-spark(/.*)?]
- </exclude>
</excludes>
</fileSet>
<!-- LICENSE, NOTICE, DEPENDENCIES, git.properties, etc. calculated at build time -->
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/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 a42b125..e8c7fae 100644
--- a/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
+++ b/assembly/src/test/java/org/apache/kylin/job/DeployUtil.java
@@ -82,7 +82,6 @@ public class DeployUtil {
config().overrideMRJobJarPath(jobJar.getAbsolutePath());
config().overrideCoprocessorLocalJar(coprocessorJar.getAbsolutePath());
- config().overrideSparkJobJarPath(getSparkJobJarFile().getAbsolutePath());
}
private static String getPomVersion() {
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/build/conf/kylin-spark-conf.properties
----------------------------------------------------------------------
diff --git a/build/conf/kylin-spark-conf.properties b/build/conf/kylin-spark-conf.properties
new file mode 100644
index 0000000..81567bb
--- /dev/null
+++ b/build/conf/kylin-spark-conf.properties
@@ -0,0 +1,26 @@
+spark.yarn.submit.file.replication=1
+spark.yarn.executor.memoryOverhead=200
+spark.yarn.driver.memoryOverhead=384
+spark.master=yarn
+spark.submit.deployMode=cluster
+spark.eventLog.enabled=true
+spark.yarn.scheduler.heartbeat.interval-ms=5000
+spark.yarn.preserve.staging.files=true
+spark.yarn.queue=default
+spark.yarn.containerLauncherMaxThreads=25
+spark.yarn.max.executor.failures=3
+spark.eventLog.dir=hdfs\:///kylin/spark-history
+spark.history.kerberos.enabled=true
+spark.history.provider=org.apache.spark.deploy.history.FsHistoryProvider
+spark.history.ui.port=18080
+spark.history.fs.logDirectory=hdfs\:///kylin/spark-history
+spark.executor.memory=4G
+spark.storage.memoryFraction=0.3
+spark.executor.cores=4
+spark.executor.instances=8
+spark.history.kerberos.keytab=none
+spark.history.kerberos.principal=none
+#spark.yarn.jar=hdfs://sandbox.hortonworks.com:8020/apps/spark/spark-assembly-1.6.3-hadoop2.6.0.jar
+spark.driver.extraJavaOptions=-Dhdp.version=current
+spark.yarn.am.extraJavaOptions=-Dhdp.version=current
+spark.executor.extraJavaOptions=-Dhdp.version=current
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/build/conf/kylin.properties
----------------------------------------------------------------------
diff --git a/build/conf/kylin.properties b/build/conf/kylin.properties
index 5e7d235..98b66cb 100644
--- a/build/conf/kylin.properties
+++ b/build/conf/kylin.properties
@@ -128,6 +128,26 @@ kylin.engine.mr.max-reducer-number=500
kylin.engine.mr.mapper-input-rows=1000000
+### Spark Engine ###
+
+# Hadoop conf folder, will export this as "HADOOP_CONF_DIR" before run spark-submit
+kylin.engine.spark.env.hadoop-conf-dir=/etc/hadoop/conf
+
+# Spark install home, default be $KYLIN_HOME/spark/
+#kylin.engine.spark.spark-home=
+
+# Spark job submission properties file, default be $KYLIN_HOME/conf/kylin-spark-conf.properties
+#kylin.engine.spark.properties-file=
+
+# Estimate the RDD partition numbers
+kylin.engine.spark.rdd-partition-cut-mb=200
+
+# Minimal partition numbers of rdd
+kylin.engine.spark.min-partition=1
+
+# Max partition numbers of rdd
+kylin.engine.spark.max-partition=500
+
### CUBE | DICTIONARY ###
# 'auto', 'inmem', 'layer' or 'random' for testing
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/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 bb8880b..9f5d4db 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
@@ -640,6 +640,7 @@ abstract public class KylinConfigBase implements Serializable {
// ref constants in IEngineAware
r.put(0, "org.apache.kylin.engine.mr.MRBatchCubingEngine");
r.put(2, "org.apache.kylin.engine.mr.MRBatchCubingEngine2");
+ r.put(4, "org.apache.kylin.engine.spark.SparkBatchCubingEngine2");
return r;
}
@@ -716,29 +717,62 @@ abstract public class KylinConfigBase implements Serializable {
// ENGINE.SPARK
// ============================================================================
- public String getKylinSparkJobJarPath() {
- final String jobJar = getOptional("kylin.engine.spark.job-jar");
- if (StringUtils.isNotEmpty(jobJar)) {
- return jobJar;
+ public String getSparkHome() {
+ String sparkHome = getOptional("kylin.engine.spark.spark-home", "spark");
+ File f = new File(sparkHome);
+ if (f.exists()) {
+ return f.getAbsolutePath();
+ } else {
+ String home = getKylinHome();
+ f = new File(home, sparkHome);
+ if (f.exists()) {
+ return f.getAbsolutePath();
+ }
}
- String kylinHome = getKylinHome();
- if (StringUtils.isEmpty(kylinHome)) {
- return "";
+
+ throw new IllegalArgumentException("Spark home '" + sparkHome + "' does not exist, check 'kylin.engine.spark.spark-home' in kylin.properties");
+
+ }
+
+ public String getSparkHadoopConfDir() {
+ return getRequired("kylin.engine.spark.env.hadoop-conf-dir");
+ }
+
+ public String getSparkConfFile() {
+ String conf = getOptional("kylin.engine.spark.properties-file", "conf/kylin-spark-conf.properties");
+ File f = new File(conf);
+ if (f.exists()) {
+ return f.getAbsolutePath();
+ } else {
+ String home = getKylinHome();
+ f = new File(home, conf);
+ if (f.exists()) {
+ return f.getAbsolutePath();
+ }
}
- return getFileName(kylinHome + File.separator + "lib", SPARK_JOB_JAR_NAME_PATTERN);
+
+ throw new IllegalArgumentException("Spark conf properties file '" + conf + "' does not exist.");
}
- public void overrideSparkJobJarPath(String path) {
- logger.info("override " + "kylin.engine.spark.job-jar" + " to " + path);
- System.setProperty("kylin.engine.spark.job-jar", path);
+ public String getSparkAdditionalJars() {
+ return getOptional("kylin.engine.spark.additional-jars", "");
}
- public String getSparkHome() {
- return getRequired("kylin.engine.spark.spark-home");
+ public float getSparkRDDPartitionCutMB() {
+ return Float.valueOf(getOptional("kylin.engine.spark.rdd-partition-cut-mb", "200.0"));
+ }
+
+
+ public int getSparkMinPartition() {
+ return Integer.valueOf(getOptional("kylin.engine.spark.min-partition", "1"));
+ }
+
+ public int getSparkMaxPartition() {
+ return Integer.valueOf(getOptional("kylin.engine.spark.max-partition", "500"));
}
- public String getSparkMaster() {
- return getRequired("kylin.engine.spark.spark-master");
+ public boolean isSparkSanityCheckEnabled() {
+ return Boolean.parseBoolean(getOptional("kylin.engine.spark.sanity-check-enabled", "false"));
}
// ============================================================================
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java b/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
index 0580576..9549569 100644
--- a/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
+++ b/core-common/src/main/java/org/apache/kylin/common/persistence/ResourceStore.java
@@ -21,17 +21,21 @@ package org.apache.kylin.common.persistence;
import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
+import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.NavigableSet;
import java.util.concurrent.ConcurrentHashMap;
+import org.apache.commons.io.FileUtils;
import org.apache.commons.io.IOUtils;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.util.ClassUtil;
+import org.apache.kylin.common.util.OptionsHelper;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -311,4 +315,35 @@ abstract public class ResourceStore {
return collector;
}
+ public static String dumpResources(KylinConfig kylinConfig, Collection<String> dumpList) throws IOException {
+ File tmp = File.createTempFile("kylin_job_meta", "");
+ FileUtils.forceDelete(tmp); // we need a directory, so delete the file first
+
+ File metaDir = new File(tmp, "meta");
+ metaDir.mkdirs();
+
+ // write kylin.properties
+ File kylinPropsFile = new File(metaDir, "kylin.properties");
+ kylinConfig.writeProperties(kylinPropsFile);
+
+ ResourceStore from = ResourceStore.getStore(kylinConfig);
+ KylinConfig localConfig = KylinConfig.createInstanceFromUri(metaDir.getAbsolutePath());
+ ResourceStore to = ResourceStore.getStore(localConfig);
+ for (String path : dumpList) {
+ RawResource res = from.getResource(path);
+ if (res == null)
+ throw new IllegalStateException("No resource found at -- " + path);
+ to.putResource(path, res.inputStream, res.timestamp);
+ res.inputStream.close();
+ }
+
+ String metaDirURI = OptionsHelper.convertToFileURL(metaDir.getAbsolutePath());
+ if (metaDirURI.startsWith("/")) // note Path on windows is like "d:/../..."
+ metaDirURI = "file://" + metaDirURI;
+ else
+ metaDirURI = "file:///" + metaDirURI;
+ logger.info("meta dir is: " + metaDirURI);
+
+ return metaDirURI;
+ }
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-common/src/main/java/org/apache/kylin/common/util/Array.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/Array.java b/core-common/src/main/java/org/apache/kylin/common/util/Array.java
index 7447b46..b25b764 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/Array.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/Array.java
@@ -23,7 +23,7 @@ import java.util.Arrays;
/*
* An array with correct equals(), hashCode(), compareTo() and toString()
*/
-public class Array<T> implements Comparable<Array<T>> {
+public class Array<T> implements Comparable<Array<T>>, java.io.Serializable {
public T[] data;
public Array(T[] data) {
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-common/src/main/java/org/apache/kylin/common/util/SplittedBytes.java
----------------------------------------------------------------------
diff --git a/core-common/src/main/java/org/apache/kylin/common/util/SplittedBytes.java b/core-common/src/main/java/org/apache/kylin/common/util/SplittedBytes.java
index 8751b78..ae380cb 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/SplittedBytes.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/SplittedBytes.java
@@ -22,7 +22,7 @@ package org.apache.kylin.common.util;
* @author George Song (ysong1)
*
*/
-public class SplittedBytes {
+public class SplittedBytes implements java.io.Serializable {
public SplittedBytes(int length) {
this.value = new byte[length];
this.length = 0;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/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 36a6044..45310f0 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
@@ -51,7 +51,7 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class CubeSegment implements Comparable<CubeSegment>, IBuildable, ISegment {
+public class CubeSegment implements Comparable<CubeSegment>, IBuildable, ISegment, java.io.Serializable {
@JsonBackReference
private CubeInstance cubeInstance;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/common/RowKeySplitter.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/common/RowKeySplitter.java b/core-cube/src/main/java/org/apache/kylin/cube/common/RowKeySplitter.java
index 67f1751..acebce4 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/common/RowKeySplitter.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/common/RowKeySplitter.java
@@ -26,9 +26,11 @@ import org.apache.kylin.cube.kv.CubeDimEncMap;
import org.apache.kylin.cube.kv.RowConstants;
import org.apache.kylin.cube.kv.RowKeyColumnIO;
import org.apache.kylin.cube.model.CubeDesc;
+import org.apache.kylin.cube.model.RowKeyColDesc;
+import org.apache.kylin.dimension.IDimensionEncodingMap;
import org.apache.kylin.metadata.model.TblColRef;
-public class RowKeySplitter {
+public class RowKeySplitter implements java.io.Serializable {
private CubeDesc cubeDesc;
private RowKeyColumnIO colIO;
@@ -37,7 +39,6 @@ public class RowKeySplitter {
private int[] splitOffsets;
private int bufferSize;
- private long lastSplittedCuboidId;
private boolean enableSharding;
private short shardId;
@@ -64,7 +65,13 @@ public class RowKeySplitter {
public RowKeySplitter(CubeSegment cubeSeg, int splitLen, int bytesLen) {
this.enableSharding = cubeSeg.isEnableSharding();
this.cubeDesc = cubeSeg.getCubeDesc();
- this.colIO = new RowKeyColumnIO(new CubeDimEncMap(cubeSeg));
+ IDimensionEncodingMap dimEncoding = new CubeDimEncMap(cubeSeg);
+
+ for (RowKeyColDesc rowKeyColDesc : cubeDesc.getRowkey().getRowKeyColumns()) {
+ dimEncoding.get(rowKeyColDesc.getColRef());
+ }
+
+ this.colIO = new RowKeyColumnIO(dimEncoding);
this.splitBuffers = new SplittedBytes[splitLen];
this.splitOffsets = new int[splitLen];
@@ -105,7 +112,7 @@ public class RowKeySplitter {
System.arraycopy(bytes, offset, cuboidIdSplit.value, 0, RowConstants.ROWKEY_CUBOIDID_LEN);
offset += RowConstants.ROWKEY_CUBOIDID_LEN;
- lastSplittedCuboidId = Bytes.toLong(cuboidIdSplit.value, 0, cuboidIdSplit.length);
+ long lastSplittedCuboidId = Bytes.toLong(cuboidIdSplit.value, 0, cuboidIdSplit.length);
Cuboid cuboid = Cuboid.findById(cubeDesc, lastSplittedCuboidId);
// rowkey columns
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
index 7503fbf..dd22d6a 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/Cuboid.java
@@ -18,15 +18,10 @@
package org.apache.kylin.cube.cuboid;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-
+import com.google.common.base.Function;
+import com.google.common.collect.Collections2;
+import com.google.common.collect.ComparisonChain;
+import com.google.common.collect.Lists;
import org.apache.commons.lang.StringUtils;
import org.apache.kylin.common.util.Bytes;
import org.apache.kylin.cube.gridtable.CuboidToGridTableMapping;
@@ -37,12 +32,16 @@ import org.apache.kylin.cube.model.RowKeyColDesc;
import org.apache.kylin.metadata.model.FunctionDesc;
import org.apache.kylin.metadata.model.TblColRef;
-import com.google.common.base.Function;
-import com.google.common.collect.Collections2;
-import com.google.common.collect.ComparisonChain;
-import com.google.common.collect.Lists;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
-public class Cuboid implements Comparable<Cuboid> {
+public class Cuboid implements Comparable<Cuboid>, java.io.Serializable {
private final static Map<String, Map<Long, Cuboid>> CUBOID_CACHE = new ConcurrentHashMap<String, Map<Long, Cuboid>>();
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidScheduler.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidScheduler.java b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidScheduler.java
index 733aded..ffb0a5e 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidScheduler.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/cuboid/CuboidScheduler.java
@@ -21,6 +21,12 @@ package org.apache.kylin.cube.cuboid;
/**
*/
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.kylin.cube.model.AggregationGroup;
+import org.apache.kylin.cube.model.CubeDesc;
+
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
@@ -28,14 +34,7 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
-import org.apache.kylin.cube.model.AggregationGroup;
-import org.apache.kylin.cube.model.CubeDesc;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-
-public class CuboidScheduler {
+public class CuboidScheduler implements java.io.Serializable {
private final CubeDesc cubeDesc;
private final long max;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
index 66a444e..b4cba39 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/inmemcubing/InMemCubeBuilder.java
@@ -462,6 +462,8 @@ public class InMemCubeBuilder extends AbstractInMemCubeBuilder {
for (int i = 0; i < totalSum.length; i++) {
if (totalSum[i] instanceof DoubleMutable) {
totalSum[i] = Math.round(((DoubleMutable) totalSum[i]).get());
+ } else if (totalSum[i] instanceof Double) {
+ totalSum[i] = Math.round(((Double) totalSum[i]).doubleValue());
} else if (totalSum[i] instanceof TopNCounter) {
TopNCounter counter = (TopNCounter) totalSum[i];
Iterator<Counter> iterator = counter.iterator();
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/kv/AbstractRowKeyEncoder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/kv/AbstractRowKeyEncoder.java b/core-cube/src/main/java/org/apache/kylin/cube/kv/AbstractRowKeyEncoder.java
index bfe6eb4..2becde4 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/kv/AbstractRowKeyEncoder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/kv/AbstractRowKeyEncoder.java
@@ -18,8 +18,6 @@
package org.apache.kylin.cube.kv;
-import java.util.Map;
-
import org.apache.kylin.common.util.ByteArray;
import org.apache.kylin.common.util.ImmutableBitSet;
import org.apache.kylin.cube.CubeSegment;
@@ -30,12 +28,14 @@ import org.apache.kylin.metadata.model.TblColRef;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.util.Map;
+
/**
*
* @author xjiang
*
*/
-public abstract class AbstractRowKeyEncoder {
+public abstract class AbstractRowKeyEncoder implements java.io.Serializable {
protected static final Logger logger = LoggerFactory.getLogger(AbstractRowKeyEncoder.class);
public static final byte DEFAULT_BLANK_BYTE = DimensionEncoding.NULL;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/kv/CubeDimEncMap.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/kv/CubeDimEncMap.java b/core-cube/src/main/java/org/apache/kylin/cube/kv/CubeDimEncMap.java
index a4d2d6f..bd9554a 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/kv/CubeDimEncMap.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/kv/CubeDimEncMap.java
@@ -18,8 +18,7 @@
package org.apache.kylin.cube.kv;
-import java.util.Map;
-
+import com.google.common.collect.Maps;
import org.apache.kylin.common.util.Dictionary;
import org.apache.kylin.cube.CubeSegment;
import org.apache.kylin.cube.model.CubeDesc;
@@ -33,9 +32,9 @@ import org.apache.kylin.metadata.model.TblColRef;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import com.google.common.collect.Maps;
+import java.util.Map;
-public class CubeDimEncMap implements IDimensionEncodingMap {
+public class CubeDimEncMap implements IDimensionEncodingMap, java.io.Serializable {
private static final Logger logger = LoggerFactory.getLogger(CubeDimEncMap.class);
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnIO.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnIO.java b/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnIO.java
index fbb93db..65911a0 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnIO.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyColumnIO.java
@@ -29,7 +29,7 @@ import org.apache.kylin.metadata.model.TblColRef;
*
* @author yangli9
*/
-public class RowKeyColumnIO {
+public class RowKeyColumnIO implements java.io.Serializable {
//private static final Logger logger = LoggerFactory.getLogger(RowKeyColumnIO.class);
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyEncoder.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyEncoder.java b/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyEncoder.java
index bf20de1..a669fb1 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyEncoder.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyEncoder.java
@@ -18,11 +18,7 @@
package org.apache.kylin.cube.kv;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
+import com.google.common.base.Preconditions;
import org.apache.kylin.common.util.ByteArray;
import org.apache.kylin.common.util.BytesUtil;
import org.apache.kylin.common.util.ImmutableBitSet;
@@ -32,9 +28,12 @@ import org.apache.kylin.cube.cuboid.Cuboid;
import org.apache.kylin.gridtable.GTRecord;
import org.apache.kylin.metadata.model.TblColRef;
-import com.google.common.base.Preconditions;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
-public class RowKeyEncoder extends AbstractRowKeyEncoder {
+public class RowKeyEncoder extends AbstractRowKeyEncoder implements java.io.Serializable {
private int bodyLength = 0;
private RowKeyColumnIO colIO;
@@ -42,11 +41,13 @@ public class RowKeyEncoder extends AbstractRowKeyEncoder {
protected boolean enableSharding;
private int uhcOffset = -1;//it's a offset to the beginning of body
private int uhcLength = -1;
+ private int headerLength;
public RowKeyEncoder(CubeSegment cubeSeg, Cuboid cuboid) {
super(cubeSeg, cuboid);
enableSharding = cubeSeg.isEnableSharding();
- Set<TblColRef> shardByColumns = cubeSeg.getShardByColumns();
+ headerLength = cubeSeg.getRowKeyPreambleSize();
+ Set<TblColRef> shardByColumns = cubeSeg.getCubeDesc().getShardByColumns();
if (shardByColumns.size() > 1) {
throw new IllegalStateException("Does not support multiple UHC now");
}
@@ -61,7 +62,7 @@ public class RowKeyEncoder extends AbstractRowKeyEncoder {
}
public int getHeaderLength() {
- return cubeSeg.getRowKeyPreambleSize();
+ return headerLength;
}
public int getBytesLength() {
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyEncoderProvider.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyEncoderProvider.java b/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyEncoderProvider.java
index 2b1dea7..791b564 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyEncoderProvider.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/kv/RowKeyEncoderProvider.java
@@ -25,7 +25,7 @@ import org.apache.kylin.cube.cuboid.Cuboid;
import com.google.common.collect.Maps;
-public class RowKeyEncoderProvider {
+public class RowKeyEncoderProvider implements java.io.Serializable {
private CubeSegment cubeSegment;
private HashMap<Long, RowKeyEncoder> rowKeyEncoders;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java b/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java
index 9bd082f..5f3c92c 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/AggregationGroup.java
@@ -35,8 +35,8 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class AggregationGroup {
- public static class HierarchyMask {
+public class AggregationGroup implements java.io.Serializable {
+ public static class HierarchyMask implements java.io.Serializable {
public long fullMask; // 00000111
public long[] allMasks; // 00000100,00000110,00000111
public long[] dims; // 00000100,00000010,00000001
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/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 4686a15..0e54b98 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
@@ -100,7 +100,7 @@ public class CubeDesc extends RootPersistentEntity implements IEngineAware {
LOOKUP, PK_FK, EXTENDED_COLUMN
}
- public static class DeriveInfo {
+ public static class DeriveInfo implements java.io.Serializable {
public DeriveType type;
public JoinDesc join;
public TblColRef[] columns;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java
index 94e1a7c..2a68116 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableDesc.java
@@ -36,7 +36,7 @@ import com.google.common.collect.Maps;
/**
*/
-public class CubeJoinedFlatTableDesc implements IJoinedFlatTableDesc {
+public class CubeJoinedFlatTableDesc implements IJoinedFlatTableDesc, java.io.Serializable {
protected final String tableName;
protected final CubeDesc cubeDesc;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableEnrich.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableEnrich.java b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableEnrich.java
index a1312b5..e829aeb 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableEnrich.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/CubeJoinedFlatTableEnrich.java
@@ -18,8 +18,6 @@
package org.apache.kylin.cube.model;
-import java.util.List;
-
import org.apache.kylin.cube.cuboid.Cuboid;
import org.apache.kylin.metadata.model.DataModelDesc;
import org.apache.kylin.metadata.model.FunctionDesc;
@@ -28,10 +26,12 @@ import org.apache.kylin.metadata.model.ISegment;
import org.apache.kylin.metadata.model.MeasureDesc;
import org.apache.kylin.metadata.model.TblColRef;
+import java.util.List;
+
/**
* An enrich of IJoinedFlatTableDesc for cubes
*/
-public class CubeJoinedFlatTableEnrich implements IJoinedFlatTableDesc {
+public class CubeJoinedFlatTableEnrich implements IJoinedFlatTableDesc, java.io.Serializable {
private CubeDesc cubeDesc;
private IJoinedFlatTableDesc flatDesc;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/model/DictionaryDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/DictionaryDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/DictionaryDesc.java
index f471f9f..ca2183a 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/DictionaryDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/DictionaryDesc.java
@@ -27,7 +27,7 @@ import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonProperty;
@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class DictionaryDesc {
+public class DictionaryDesc implements java.io.Serializable {
@JsonProperty("column")
private String column;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/model/DimensionDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/DimensionDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/DimensionDesc.java
index cd75228..3488c36 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/DimensionDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/DimensionDesc.java
@@ -20,7 +20,6 @@ package org.apache.kylin.cube.model;
import java.util.Arrays;
-import org.apache.commons.lang.NotImplementedException;
import org.apache.kylin.metadata.model.DataModelDesc;
import org.apache.kylin.metadata.model.JoinDesc;
import org.apache.kylin.metadata.model.JoinTableDesc;
@@ -35,7 +34,7 @@ import com.google.common.base.Objects;
/**
*/
@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class DimensionDesc {
+public class DimensionDesc implements java.io.Serializable {
@JsonProperty("name")
private String name;
@@ -139,16 +138,6 @@ public class DimensionDesc {
}
@Override
- public boolean equals(Object o) {
- throw new NotImplementedException();
- }
-
- @Override
- public int hashCode() {
- throw new NotImplementedException();
- }
-
- @Override
public String toString() {
return Objects.toStringHelper(this).add("name", name).add("table", table).add("column", column).add("derived", Arrays.toString(derived)).add("join", join).toString();
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseColumnDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseColumnDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseColumnDesc.java
index fb491f8..7007342 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseColumnDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseColumnDesc.java
@@ -18,19 +18,18 @@
package org.apache.kylin.cube.model;
-import java.util.Arrays;
-
-import org.apache.kylin.metadata.model.FunctionDesc;
-import org.apache.kylin.metadata.model.MeasureDesc;
-
import com.fasterxml.jackson.annotation.JsonAutoDetect;
import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kylin.metadata.model.FunctionDesc;
+import org.apache.kylin.metadata.model.MeasureDesc;
+
+import java.util.Arrays;
/**
*/
@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class HBaseColumnDesc {
+public class HBaseColumnDesc implements java.io.Serializable {
@JsonProperty("qualifier")
private String qualifier;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseColumnFamilyDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseColumnFamilyDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseColumnFamilyDesc.java
index c5b2e19..85c2c17 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseColumnFamilyDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseColumnFamilyDesc.java
@@ -18,18 +18,17 @@
package org.apache.kylin.cube.model;
-import java.util.Arrays;
-
-import org.apache.kylin.metadata.model.MeasureDesc;
-
import com.fasterxml.jackson.annotation.JsonAutoDetect;
import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.kylin.metadata.model.MeasureDesc;
+
+import java.util.Arrays;
/**
*/
@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class HBaseColumnFamilyDesc {
+public class HBaseColumnFamilyDesc implements java.io.Serializable {
@JsonProperty("name")
private String name;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseMappingDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseMappingDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseMappingDesc.java
index 2ef1e17..9ad8407 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseMappingDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/HBaseMappingDesc.java
@@ -18,22 +18,21 @@
package org.apache.kylin.cube.model;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.LinkedList;
-
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.kylin.common.util.StringUtil;
import org.apache.kylin.metadata.model.FunctionDesc;
import org.apache.kylin.metadata.model.MeasureDesc;
-import com.fasterxml.jackson.annotation.JsonAutoDetect;
-import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
-import com.fasterxml.jackson.annotation.JsonProperty;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.LinkedList;
/**
*/
@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class HBaseMappingDesc {
+public class HBaseMappingDesc implements java.io.Serializable {
@JsonProperty("column_family")
private HBaseColumnFamilyDesc[] columnFamily;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/model/HierarchyDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/HierarchyDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/HierarchyDesc.java
index f88d4d2..b72f220 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/HierarchyDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/HierarchyDesc.java
@@ -27,7 +27,7 @@ import com.fasterxml.jackson.annotation.JsonProperty;
/**
*/
@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class HierarchyDesc {
+public class HierarchyDesc implements java.io.Serializable {
@JsonProperty("level")
private String level;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
index ef34a9b..71fd4b9 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyColDesc.java
@@ -40,7 +40,7 @@ import com.google.common.base.Preconditions;
*
*/
@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class RowKeyColDesc {
+public class RowKeyColDesc implements java.io.Serializable {
@JsonProperty("column")
private String column;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java
index f1a403d..00557c5 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/RowKeyDesc.java
@@ -18,24 +18,23 @@
package org.apache.kylin.cube.model;
+import com.fasterxml.jackson.annotation.JsonAutoDetect;
+import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Objects;
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.kylin.metadata.model.TblColRef;
+
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
-import org.apache.commons.lang.ArrayUtils;
-import org.apache.kylin.metadata.model.TblColRef;
-
-import com.fasterxml.jackson.annotation.JsonAutoDetect;
-import com.fasterxml.jackson.annotation.JsonAutoDetect.Visibility;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Objects;
-
/**
*/
@JsonAutoDetect(fieldVisibility = Visibility.NONE, getterVisibility = Visibility.NONE, isGetterVisibility = Visibility.NONE, setterVisibility = Visibility.NONE)
-public class RowKeyDesc {
+public class RowKeyDesc implements java.io.Serializable {
@JsonProperty("rowkey_columns")
private RowKeyColDesc[] rowkeyColumns;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/cube/model/SelectRule.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/cube/model/SelectRule.java b/core-cube/src/main/java/org/apache/kylin/cube/model/SelectRule.java
index 63b0fc4..4a6c510 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/model/SelectRule.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/model/SelectRule.java
@@ -20,7 +20,7 @@ package org.apache.kylin.cube.model;
import com.fasterxml.jackson.annotation.JsonProperty;
-public class SelectRule {
+public class SelectRule implements java.io.Serializable {
@JsonProperty("hierarchy_dims")
public String[][] hierarchy_dims;
@JsonProperty("mandatory_dims")
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java b/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java
index b8d116c..6190f29 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/UnitTestSupport.java
@@ -28,7 +28,6 @@ import org.apache.kylin.common.util.ImmutableBitSet;
import org.apache.kylin.gridtable.GTInfo.Builder;
import org.apache.kylin.measure.hllc.HLLCounter;
import org.apache.kylin.metadata.datatype.DataType;
-import org.apache.kylin.metadata.datatype.LongMutable;
public class UnitTestSupport {
@@ -84,16 +83,16 @@ public class UnitTestSupport {
String d_01_15 = datePlus("2015-01-15", i * 4);
String d_01_16 = datePlus("2015-01-16", i * 4);
String d_01_17 = datePlus("2015-01-17", i * 4);
- result.add(newRec(info, d_01_14, "Yang", "Food", new LongMutable(10), new BigDecimal("10.5")));
- result.add(newRec(info, d_01_14, "Luke", "Food", new LongMutable(10), new BigDecimal("10.5")));
- result.add(newRec(info, d_01_15, "Xu", "Food", new LongMutable(10), new BigDecimal("10.5")));
- result.add(newRec(info, d_01_15, "Dong", "Food", new LongMutable(10), new BigDecimal("10.5")));
- result.add(newRec(info, d_01_15, "Jason", "Food", new LongMutable(10), new BigDecimal("10.5")));
- result.add(newRec(info, d_01_16, "Mahone", "Food", new LongMutable(10), new BigDecimal("10.5")));
- result.add(newRec(info, d_01_16, "Shaofeng", "Food", new LongMutable(10), new BigDecimal("10.5")));
- result.add(newRec(info, d_01_16, "Qianhao", "Food", new LongMutable(10), new BigDecimal("10.5")));
- result.add(newRec(info, d_01_16, "George", "Food", new LongMutable(10), new BigDecimal("10.5")));
- result.add(newRec(info, d_01_17, "Kejia", "Food", new LongMutable(10), new BigDecimal("10.5")));
+ result.add(newRec(info, d_01_14, "Yang", "Food", new Long(10), new BigDecimal("10.5")));
+ result.add(newRec(info, d_01_14, "Luke", "Food", new Long(10), new BigDecimal("10.5")));
+ result.add(newRec(info, d_01_15, "Xu", "Food", new Long(10), new BigDecimal("10.5")));
+ result.add(newRec(info, d_01_15, "Dong", "Food", new Long(10), new BigDecimal("10.5")));
+ result.add(newRec(info, d_01_15, "Jason", "Food", new Long(10), new BigDecimal("10.5")));
+ result.add(newRec(info, d_01_16, "Mahone", "Food", new Long(10), new BigDecimal("10.5")));
+ result.add(newRec(info, d_01_16, "Shaofeng", "Food", new Long(10), new BigDecimal("10.5")));
+ result.add(newRec(info, d_01_16, "Qianhao", "Food", new Long(10), new BigDecimal("10.5")));
+ result.add(newRec(info, d_01_16, "George", "Food", new Long(10), new BigDecimal("10.5")));
+ result.add(newRec(info, d_01_17, "Kejia", "Food", new Long(10), new BigDecimal("10.5")));
}
return result;
}
@@ -106,16 +105,16 @@ public class UnitTestSupport {
String d_01_15 = datePlus("2015-01-15", i * 4);
String d_01_16 = datePlus("2015-01-16", i * 4);
String d_01_17 = datePlus("2015-01-17", i * 4);
- result.add(newRec(info, d_01_14, "Yang", "Food", new LongMutable(10), new BigDecimal("10.5"), new HLLCounter(14)));
- result.add(newRec(info, d_01_14, "Luke", "Food", new LongMutable(10), new BigDecimal("10.5"), new HLLCounter(14)));
- result.add(newRec(info, d_01_15, "Xu", "Food", new LongMutable(10), new BigDecimal("10.5"), new HLLCounter(14)));
- result.add(newRec(info, d_01_15, "Dong", "Food", new LongMutable(10), new BigDecimal("10.5"), new HLLCounter(14)));
- result.add(newRec(info, d_01_15, "Jason", "Food", new LongMutable(10), new BigDecimal("10.5"), new HLLCounter(14)));
- result.add(newRec(info, d_01_16, "Mahone", "Food", new LongMutable(10), new BigDecimal("10.5"), new HLLCounter(14)));
- result.add(newRec(info, d_01_16, "Shaofeng", "Food", new LongMutable(10), new BigDecimal("10.5"), new HLLCounter(14)));
- result.add(newRec(info, d_01_16, "Qianhao", "Food", new LongMutable(10), new BigDecimal("10.5"), new HLLCounter(14)));
- result.add(newRec(info, d_01_16, "George", "Food", new LongMutable(10), new BigDecimal("10.5"), new HLLCounter(14)));
- result.add(newRec(info, d_01_17, "Kejia", "Food", new LongMutable(10), new BigDecimal("10.5"), new HLLCounter(14)));
+ result.add(newRec(info, d_01_14, "Yang", "Food", new Long(10), new BigDecimal("10.5"), new HLLCounter(14)));
+ result.add(newRec(info, d_01_14, "Luke", "Food", new Long(10), new BigDecimal("10.5"), new HLLCounter(14)));
+ result.add(newRec(info, d_01_15, "Xu", "Food", new Long(10), new BigDecimal("10.5"), new HLLCounter(14)));
+ result.add(newRec(info, d_01_15, "Dong", "Food", new Long(10), new BigDecimal("10.5"), new HLLCounter(14)));
+ result.add(newRec(info, d_01_15, "Jason", "Food", new Long(10), new BigDecimal("10.5"), new HLLCounter(14)));
+ result.add(newRec(info, d_01_16, "Mahone", "Food", new Long(10), new BigDecimal("10.5"), new HLLCounter(14)));
+ result.add(newRec(info, d_01_16, "Shaofeng", "Food", new Long(10), new BigDecimal("10.5"), new HLLCounter(14)));
+ result.add(newRec(info, d_01_16, "Qianhao", "Food", new Long(10), new BigDecimal("10.5"), new HLLCounter(14)));
+ result.add(newRec(info, d_01_16, "George", "Food", new Long(10), new BigDecimal("10.5"), new HLLCounter(14)));
+ result.add(newRec(info, d_01_17, "Kejia", "Food", new Long(10), new BigDecimal("10.5"), new HLLCounter(14)));
}
return result;
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
index 3807b70..b164e54 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheMemSizeTest.java
@@ -36,8 +36,6 @@ import org.apache.kylin.measure.bitmap.BitmapAggregator;
import org.apache.kylin.measure.bitmap.BitmapCounter;
import org.apache.kylin.measure.hllc.HLLCAggregator;
import org.apache.kylin.measure.hllc.HLLCounter;
-import org.apache.kylin.metadata.datatype.DoubleMutable;
-import org.apache.kylin.metadata.datatype.LongMutable;
import org.github.jamm.MemoryMeter;
import org.junit.Test;
@@ -100,10 +98,10 @@ public class AggregationCacheMemSizeTest {
private MeasureAggregator<?>[] createNoMemHungryAggrs() {
LongSumAggregator longSum = new LongSumAggregator();
- longSum.aggregate(new LongMutable(10));
+ longSum.aggregate(new Long(10));
DoubleSumAggregator doubleSum = new DoubleSumAggregator();
- doubleSum.aggregate(new DoubleMutable(10));
+ doubleSum.aggregate(new Double(10));
BigDecimalSumAggregator decimalSum = new BigDecimalSumAggregator();
decimalSum.aggregate(new BigDecimal("12345678901234567890.123456789"));
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheSpillTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheSpillTest.java b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheSpillTest.java
index d7787e0..74c2659 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheSpillTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/AggregationCacheSpillTest.java
@@ -28,7 +28,6 @@ import java.util.List;
import org.apache.kylin.common.util.ImmutableBitSet;
import org.apache.kylin.common.util.LocalFileMetadataTestCase;
-import org.apache.kylin.metadata.datatype.LongMutable;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -91,7 +90,7 @@ public class AggregationCacheSpillTest extends LocalFileMetadataTestCase {
for (GTRecord record : scanner) {
assertNotNull(record);
Object[] returnRecord = record.getValues();
- assertEquals(20, ((LongMutable) returnRecord[3]).get());
+ assertEquals(20, ((Long) returnRecord[3]).longValue());
assertEquals(21, ((BigDecimal) returnRecord[4]).longValue());
count++;
@@ -133,7 +132,7 @@ public class AggregationCacheSpillTest extends LocalFileMetadataTestCase {
for (GTRecord record : scanner) {
assertNotNull(record);
Object[] returnRecord = record.getValues();
- assertEquals(80000, ((LongMutable) returnRecord[3]).get());
+ assertEquals(80000, ((Long) returnRecord[3]).longValue());
assertEquals(84000, ((BigDecimal) returnRecord[4]).longValue());
count++;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleGridTableTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleGridTableTest.java b/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleGridTableTest.java
index f315b1e..4ac6644 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleGridTableTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleGridTableTest.java
@@ -29,7 +29,6 @@ import java.util.List;
import org.apache.kylin.common.util.ImmutableBitSet;
import org.apache.kylin.common.util.LocalFileMetadataTestCase;
import org.apache.kylin.gridtable.memstore.GTSimpleMemStore;
-import org.apache.kylin.metadata.datatype.LongMutable;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@@ -97,7 +96,7 @@ public class SimpleGridTableTest extends LocalFileMetadataTestCase {
Object[] v = r.getValues();
assertTrue(((String) v[0]).startsWith("2015-"));
assertTrue(((String) v[2]).equals("Food"));
- assertTrue(((LongMutable) v[3]).get() == 10);
+ assertTrue(((Long) v[3]).longValue() == 10);
assertTrue(((BigDecimal) v[4]).doubleValue() == 10.5);
System.out.println(r);
}
@@ -114,19 +113,19 @@ public class SimpleGridTableTest extends LocalFileMetadataTestCase {
Object[] v = r.getValues();
switch (i) {
case 0:
- assertTrue(((LongMutable) v[3]).get() == 20);
+ assertTrue(((Long) v[3]).longValue() == 20);
assertTrue(((BigDecimal) v[4]).doubleValue() == 21.0);
break;
case 1:
- assertTrue(((LongMutable) v[3]).get() == 30);
+ assertTrue(((Long) v[3]).longValue() == 30);
assertTrue(((BigDecimal) v[4]).doubleValue() == 31.5);
break;
case 2:
- assertTrue(((LongMutable) v[3]).get() == 40);
+ assertTrue(((Long) v[3]).longValue() == 40);
assertTrue(((BigDecimal) v[4]).doubleValue() == 42.0);
break;
case 3:
- assertTrue(((LongMutable) v[3]).get() == 10);
+ assertTrue(((Long) v[3]).longValue() == 10);
assertTrue(((BigDecimal) v[4]).doubleValue() == 10.5);
break;
default:
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleInvertedIndexTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleInvertedIndexTest.java b/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleInvertedIndexTest.java
index 2cf58ac..5e3e771 100644
--- a/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleInvertedIndexTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/gridtable/SimpleInvertedIndexTest.java
@@ -27,7 +27,6 @@ import java.util.ArrayList;
import org.apache.kylin.common.util.ByteArray;
import org.apache.kylin.common.util.LocalFileMetadataTestCase;
import org.apache.kylin.metadata.datatype.DataType;
-import org.apache.kylin.metadata.datatype.LongMutable;
import org.apache.kylin.metadata.datatype.StringSerializer;
import org.apache.kylin.metadata.filter.ColumnTupleFilter;
import org.apache.kylin.metadata.filter.CompareTupleFilter;
@@ -72,7 +71,7 @@ public class SimpleInvertedIndexTest extends LocalFileMetadataTestCase {
GTRowBlock.Writer writer = mockBlock.getWriter();
GTRecord record = new GTRecord(info);
for (int i = 0; i < 10; i++) {
- record.setValues(i < 9 ? "" + i : null, "", "", new LongMutable(0), new BigDecimal(0));
+ record.setValues(i < 9 ? "" + i : null, "", "", new Long(0), new BigDecimal(0));
for (int j = 0; j < info.getRowBlockSize(); j++) {
writer.append(record);
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
----------------------------------------------------------------------
diff --git a/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java b/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
index fc20418..ad4d90b 100644
--- a/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
+++ b/core-cube/src/test/java/org/apache/kylin/metadata/measure/MeasureCodecTest.java
@@ -27,8 +27,6 @@ import org.apache.kylin.common.util.LocalFileMetadataTestCase;
import org.apache.kylin.measure.BufferedMeasureCodec;
import org.apache.kylin.measure.bitmap.BitmapCounter;
import org.apache.kylin.measure.hllc.HLLCounter;
-import org.apache.kylin.metadata.datatype.DoubleMutable;
-import org.apache.kylin.metadata.datatype.LongMutable;
import org.apache.kylin.metadata.model.FunctionDesc;
import org.apache.kylin.metadata.model.MeasureDesc;
import org.junit.AfterClass;
@@ -54,8 +52,8 @@ public class MeasureCodecTest extends LocalFileMetadataTestCase {
MeasureDesc[] descs = new MeasureDesc[] { measure("double"), measure("long"), measure("decimal"), measure("HLLC16"), measure("bitmap") };
BufferedMeasureCodec codec = new BufferedMeasureCodec(descs);
- DoubleMutable d = new DoubleMutable(1.0);
- LongMutable l = new LongMutable(2);
+ Double d = new Double(1.0);
+ Long l = new Long(2);
BigDecimal b = new BigDecimal("333.1234");
HLLCounter hllc = new HLLCounter(16);
hllc.add("1234567");
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
index 503c29e..5029bc4 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/AppendTrieDictionary.java
@@ -133,7 +133,7 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
}
}
- public static class DictSliceKey implements WritableComparable {
+ public static class DictSliceKey implements WritableComparable, java.io.Serializable {
byte[] key;
public static DictSliceKey wrap(byte[] key) {
@@ -174,7 +174,7 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
}
}
- public static class DictSlice<T> implements Writable {
+ public static class DictSlice<T> implements Writable, java.io.Serializable {
public DictSlice() {
}
@@ -438,7 +438,7 @@ public class AppendTrieDictionary<T> extends CacheDictionary<T> {
}
}
- public static class DictNode implements Writable {
+ public static class DictNode implements Writable, java.io.Serializable {
public byte[] part;
public int id = -1;
public boolean isEndOfValue;
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java
index f1b1b3d..9458e9d 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary.java
@@ -153,7 +153,7 @@ public class NumberDictionary<T> extends TrieDictionary<T> {
}
}
- static ThreadLocal<NumberBytesCodec> localCodec = new ThreadLocal<NumberBytesCodec>();
+ static transient ThreadLocal<NumberBytesCodec> localCodec = new ThreadLocal<NumberBytesCodec>();
// ============================================================================
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary2.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary2.java b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary2.java
index b5ce1ff..80e9940 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary2.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionary2.java
@@ -24,7 +24,7 @@ package org.apache.kylin.dict;
@SuppressWarnings("serial")
public class NumberDictionary2<T> extends NumberDictionary<T> {
- static ThreadLocal<NumberBytesCodec> localCodec = new ThreadLocal<NumberBytesCodec>();
+ static transient ThreadLocal<NumberBytesCodec> localCodec = new ThreadLocal<NumberBytesCodec>();
// ============================================================================
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryForestBuilder.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryForestBuilder.java b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryForestBuilder.java
index 5502a74..7c750f4 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryForestBuilder.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/NumberDictionaryForestBuilder.java
@@ -26,10 +26,10 @@ import org.apache.kylin.dict.NumberDictionary.NumberBytesCodec;
*/
public class NumberDictionaryForestBuilder extends TrieDictionaryForestBuilder<String> {
- public static class Number2BytesConverter implements BytesConverter<String> {
+ public static class Number2BytesConverter implements BytesConverter<String>, java.io.Serializable {
static final int MAX_DIGITS_BEFORE_DECIMAL_POINT = NumberDictionary.MAX_DIGITS_BEFORE_DECIMAL_POINT;
- static final ThreadLocal<NumberBytesCodec> LOCAL = new ThreadLocal<NumberBytesCodec>();
+ static final transient ThreadLocal<NumberBytesCodec> LOCAL = new ThreadLocal<NumberBytesCodec>();
static NumberBytesCodec getCodec() {
NumberBytesCodec codec = LOCAL.get();
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-dictionary/src/main/java/org/apache/kylin/dict/StringBytesConverter.java
----------------------------------------------------------------------
diff --git a/core-dictionary/src/main/java/org/apache/kylin/dict/StringBytesConverter.java b/core-dictionary/src/main/java/org/apache/kylin/dict/StringBytesConverter.java
index 0bec6a1..9107a4c 100644
--- a/core-dictionary/src/main/java/org/apache/kylin/dict/StringBytesConverter.java
+++ b/core-dictionary/src/main/java/org/apache/kylin/dict/StringBytesConverter.java
@@ -20,7 +20,7 @@ package org.apache.kylin.dict;
import org.apache.kylin.common.util.Bytes;
-public class StringBytesConverter implements BytesConverter<String> {
+public class StringBytesConverter implements BytesConverter<String>, java.io.Serializable {
@Override
public byte[] convertToBytes(String v) {
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableManager.java
----------------------------------------------------------------------
diff --git a/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableManager.java b/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableManager.java
index 0273fd8..466cdad 100644
--- a/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableManager.java
+++ b/core-job/src/main/java/org/apache/kylin/job/execution/ExecutableManager.java
@@ -18,12 +18,9 @@
package org.apache.kylin.job.execution;
-import java.lang.reflect.Constructor;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
import org.apache.kylin.common.KylinConfig;
import org.apache.kylin.common.util.ClassUtil;
import org.apache.kylin.job.dao.ExecutableDao;
@@ -34,9 +31,11 @@ import org.apache.kylin.job.exception.PersistentException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
+import java.lang.reflect.Constructor;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
/**
*/
@@ -352,6 +351,10 @@ public class ExecutableManager {
}
}
}
+
+ if (job.getStatus() == ExecutableState.SUCCEED) {
+ updateJobOutput(job.getId(), ExecutableState.READY, null, null);
+ }
}
public void pauseJob(String jobId) {
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/dimension/AbstractDateDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/AbstractDateDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/AbstractDateDimEnc.java
index ec6347f..a54bcda 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/AbstractDateDimEnc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/AbstractDateDimEnc.java
@@ -18,6 +18,10 @@
package org.apache.kylin.dimension;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.common.util.DateFormat;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+
import java.io.IOException;
import java.io.ObjectInput;
import java.io.ObjectOutput;
@@ -25,10 +29,6 @@ import java.io.Serializable;
import java.nio.ByteBuffer;
import java.util.Arrays;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.common.util.DateFormat;
-import org.apache.kylin.metadata.datatype.DataTypeSerializer;
-
public class AbstractDateDimEnc extends DimensionEncoding {
private static final long serialVersionUID = 1L;
@@ -81,11 +81,9 @@ public class AbstractDateDimEnc extends DimensionEncoding {
@Override
public DataTypeSerializer<Object> asDataTypeSerializer() {
return new DataTypeSerializer<Object>() {
- // be thread-safe and avoid repeated obj creation
- private ThreadLocal<byte[]> current = new ThreadLocal<byte[]>();
private byte[] currentBuf() {
- byte[] buf = current.get();
+ byte[] buf = (byte[]) current.get();
if (buf == null) {
buf = new byte[fixedLen];
current.set(buf);
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/dimension/BooleanDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/BooleanDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/BooleanDimEnc.java
index c3f4c11..75e50a1 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/BooleanDimEnc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/BooleanDimEnc.java
@@ -109,11 +109,9 @@ public class BooleanDimEnc extends DimensionEncoding {
}
private class BooleanSerializer extends DataTypeSerializer<Object> {
- // be thread-safe and avoid repeated obj creation
- private ThreadLocal<byte[]> current = new ThreadLocal<byte[]>();
private byte[] currentBuf() {
- byte[] buf = current.get();
+ byte[] buf = (byte[]) current.get();
if (buf == null) {
buf = new byte[fixedLen];
current.set(buf);
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/dimension/DictionaryDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/DictionaryDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/DictionaryDimEnc.java
index 500b410..48238dc 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/DictionaryDimEnc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/DictionaryDimEnc.java
@@ -39,12 +39,15 @@ public class DictionaryDimEnc extends DimensionEncoding {
// ============================================================================
// could use a lazy loading trick here, to prevent loading all dictionaries of a segment at once
- private final Dictionary<String> dict;
- private final int fixedLen;
+ private Dictionary<String> dict;
+ private int fixedLen;
// used in encode(), when a value does not exist in dictionary
- private final int roundingFlag;
- private final byte defaultByte;
+ private int roundingFlag;
+ private byte defaultByte;
+
+ public DictionaryDimEnc() {
+ }
public DictionaryDimEnc(Dictionary<String> dict) {
this(dict, 0, NULL);
@@ -145,12 +148,18 @@ public class DictionaryDimEnc extends DimensionEncoding {
@Override
public void writeExternal(ObjectOutput out) throws IOException {
- throw new UnsupportedOperationException();
+ out.writeInt(fixedLen);
+ out.writeInt(roundingFlag);
+ out.write(defaultByte);
+ out.writeObject(dict);
}
@Override
public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
- throw new UnsupportedOperationException();
+ this.fixedLen = in.readInt();
+ this.roundingFlag = in.readInt();
+ this.defaultByte = in.readByte();
+ this.dict = (Dictionary<String>) in.readObject();
}
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenDimEnc.java
index b219766..f7f02a0 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenDimEnc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenDimEnc.java
@@ -129,11 +129,9 @@ public class FixedLenDimEnc extends DimensionEncoding {
}
public class FixedLenSerializer extends DataTypeSerializer<Object> {
- // be thread-safe and avoid repeated obj creation
- private ThreadLocal<byte[]> current = new ThreadLocal<byte[]>();
private byte[] currentBuf() {
- byte[] buf = current.get();
+ byte[] buf = (byte[]) current.get();
if (buf == null) {
buf = new byte[fixedLen];
current.set(buf);
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenHexDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenHexDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenHexDimEnc.java
index 83118fc..f90a40e 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenHexDimEnc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/FixedLenHexDimEnc.java
@@ -224,11 +224,9 @@ public class FixedLenHexDimEnc extends DimensionEncoding {
}
public class FixedLenSerializer extends DataTypeSerializer<Object> {
- // be thread-safe and avoid repeated obj creation
- private ThreadLocal<byte[]> current = new ThreadLocal<byte[]>();
private byte[] currentBuf() {
- byte[] buf = current.get();
+ byte[] buf = (byte[]) current.get();
if (buf == null) {
buf = new byte[bytelen];
current.set(buf);
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/dimension/IntDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/IntDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/IntDimEnc.java
index f25f2a6..aa954da 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/IntDimEnc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/IntDimEnc.java
@@ -109,11 +109,9 @@ public class IntDimEnc extends DimensionEncoding {
}
public class IntegerSerializer extends DataTypeSerializer<Object> {
- // be thread-safe and avoid repeated obj creation
- private ThreadLocal<byte[]> current = new ThreadLocal<byte[]>();
private byte[] currentBuf() {
- byte[] buf = current.get();
+ byte[] buf = (byte[]) current.get();
if (buf == null) {
buf = new byte[fixedLen];
current.set(buf);
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java
index 090dc83..0875a7f 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/IntegerDimEnc.java
@@ -135,11 +135,9 @@ public class IntegerDimEnc extends DimensionEncoding {
}
public class IntegerSerializer extends DataTypeSerializer<Object> {
- // be thread-safe and avoid repeated obj creation
- private ThreadLocal<byte[]> current = new ThreadLocal<byte[]>();
private byte[] currentBuf() {
- byte[] buf = current.get();
+ byte[] buf = (byte[]) current.get();
if (buf == null) {
buf = new byte[fixedLen];
current.set(buf);
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/dimension/OneMoreByteVLongDimEnc.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/dimension/OneMoreByteVLongDimEnc.java b/core-metadata/src/main/java/org/apache/kylin/dimension/OneMoreByteVLongDimEnc.java
index 993aac3..c0c52d1 100644
--- a/core-metadata/src/main/java/org/apache/kylin/dimension/OneMoreByteVLongDimEnc.java
+++ b/core-metadata/src/main/java/org/apache/kylin/dimension/OneMoreByteVLongDimEnc.java
@@ -18,17 +18,17 @@
package org.apache.kylin.dimension;
+import org.apache.kylin.common.util.BytesUtil;
+import org.apache.kylin.metadata.datatype.DataTypeSerializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
import java.io.IOException;
import java.io.ObjectInput;
import java.io.ObjectOutput;
import java.nio.ByteBuffer;
import java.util.Arrays;
-import org.apache.kylin.common.util.BytesUtil;
-import org.apache.kylin.metadata.datatype.DataTypeSerializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
/**
* not being used yet, prepared for future
*/
@@ -118,11 +118,9 @@ public class OneMoreByteVLongDimEnc extends DimensionEncoding {
}
public class VLongSerializer extends DataTypeSerializer<Object> {
- // be thread-safe and avoid repeated obj creation
- private ThreadLocal<byte[]> current = new ThreadLocal<byte[]>();
private byte[] currentBuf() {
- byte[] buf = current.get();
+ byte[] buf = (byte[]) current.get();
if (buf == null) {
buf = new byte[byteLen];
current.set(buf);
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/BufferedMeasureCodec.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/BufferedMeasureCodec.java b/core-metadata/src/main/java/org/apache/kylin/measure/BufferedMeasureCodec.java
index 8a5481c..44e5708 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/BufferedMeasureCodec.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/BufferedMeasureCodec.java
@@ -18,26 +18,26 @@
package org.apache.kylin.measure;
+import org.apache.kylin.metadata.datatype.DataType;
+import org.apache.kylin.metadata.model.MeasureDesc;
+
import java.nio.BufferOverflowException;
import java.nio.ByteBuffer;
import java.util.Collection;
-import org.apache.kylin.metadata.datatype.DataType;
-import org.apache.kylin.metadata.model.MeasureDesc;
-
/**
* This class embeds a reusable byte buffer for measure encoding, and is not thread-safe.
* The buffer will grow to accommodate BufferOverflowException until a limit.
* The problem here to solve is some measure type cannot provide accurate DataTypeSerializer.maxLength()
*/
@SuppressWarnings({ "unchecked" })
-public class BufferedMeasureCodec {
+public class BufferedMeasureCodec implements java.io.Serializable {
public static final int DEFAULT_BUFFER_SIZE = 1024 * 1024; // 1 MB
public static final int MAX_BUFFER_SIZE = 1 * 1024 * DEFAULT_BUFFER_SIZE; // 1 GB
final private MeasureCodec codec;
- private ByteBuffer buf;
+ private transient ByteBuffer buf;
final private int[] measureSizes;
public BufferedMeasureCodec(Collection<MeasureDesc> measureDescs) {
http://git-wip-us.apache.org/repos/asf/kylin/blob/a69a3b61/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregator.java
----------------------------------------------------------------------
diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregator.java b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregator.java
index 7b74225..929eff0 100644
--- a/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregator.java
+++ b/core-metadata/src/main/java/org/apache/kylin/measure/MeasureAggregator.java
@@ -71,6 +71,8 @@ abstract public class MeasureAggregator<V> implements Serializable {
abstract public void aggregate(V value);
+ abstract public V aggregate(V value1, V value2);
+
abstract public V getState();
// get an estimate of memory consumption UPPER BOUND
[05/17] kylin git commit: KYLIN-2335,
http request always use cache data in IE10 or IE11
Posted by sh...@apache.org.
KYLIN-2335,http request always use cache data in IE10 or IE11
Signed-off-by: zhongjian <ji...@163.com>
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/89e4b53d
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/89e4b53d
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/89e4b53d
Branch: refs/heads/sparkcubing-rebase
Commit: 89e4b53daaeadda3945bc796a09ee9e8dd90de9c
Parents: 5be4bba
Author: luguosheng <55...@qq.com>
Authored: Thu Dec 29 14:13:24 2016 +0800
Committer: zhongjian <ji...@163.com>
Committed: Thu Jan 5 11:26:30 2017 +0800
----------------------------------------------------------------------
webapp/app/js/config.js | 4 +++-
1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/89e4b53d/webapp/app/js/config.js
----------------------------------------------------------------------
diff --git a/webapp/app/js/config.js b/webapp/app/js/config.js
index 95cfc1a..5f72eb2 100644
--- a/webapp/app/js/config.js
+++ b/webapp/app/js/config.js
@@ -38,7 +38,9 @@ var Config = {
// Angular module to load routes.
KylinApp.config(function ($routeProvider, $httpProvider, $locationProvider, $logProvider) {
-
+ //resolve http always use cache data in IE11,IE10
+ $httpProvider.defaults.headers.common['Cache-Control'] = 'no-cache';
+ $httpProvider.defaults.headers.common['Pragma'] = 'no-cache';
// Set debug to true by default.
if (angular.isUndefined(Config.debug) || Config.debug !== false) {
Config.debug = true;