You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kylin.apache.org by li...@apache.org on 2017/03/12 02:50:49 UTC
kylin git commit: KYLIN-2497 optimize SelfDefineSortableKey (by
xiefan)
Repository: kylin
Updated Branches:
refs/heads/master 181b6c5c1 -> 77fe58136
KYLIN-2497 optimize SelfDefineSortableKey (by xiefan)
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/77fe5813
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/77fe5813
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/77fe5813
Branch: refs/heads/master
Commit: 77fe58136e0a00a64f9149e3d79532d92eeb77aa
Parents: 181b6c5
Author: Hongbin Ma <ma...@apache.org>
Authored: Sun Mar 12 10:50:24 2017 +0800
Committer: Hongbin Ma <ma...@apache.org>
Committed: Sun Mar 12 10:50:24 2017 +0800
----------------------------------------------------------------------
.../apache/kylin/engine/mr/KylinReducer.java | 2 +-
.../mr/steps/FactDistinctColumnsMapper.java | 18 ++--
.../mr/steps/FactDistinctColumnsMapperBase.java | 2 +-
.../engine/mr/steps/SelfDefineSortableKey.java | 106 +++++++++----------
.../mr/steps/NumberDictionaryForestTest.java | 19 ++--
.../mr/steps/SelfDefineSortableKeyTest.java | 82 +++++++++++---
6 files changed, 139 insertions(+), 90 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kylin/blob/77fe5813/engine-mr/src/main/java/org/apache/kylin/engine/mr/KylinReducer.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/KylinReducer.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/KylinReducer.java
index d428757..e47af9c 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/KylinReducer.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/KylinReducer.java
@@ -42,7 +42,7 @@ public class KylinReducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT> extends Reducer<KEYI
final public void reduce(KEYIN key, Iterable<VALUEIN> values, Reducer<KEYIN, VALUEIN, KEYOUT, VALUEOUT>.Context context) throws IOException, InterruptedException {
try {
if (reduceCounter++ % BatchConstants.NORMAL_RECORD_LOG_THRESHOLD == 0) {
- logger.info("Accepting Mapper Key with ordinal: " + reduceCounter);
+ logger.info("Accepting Reducer Key with ordinal: " + reduceCounter);
}
doReduce(key, values, context);
http://git-wip-us.apache.org/repos/asf/kylin/blob/77fe5813/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java
index 07e788c..9f65163 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapper.java
@@ -31,6 +31,7 @@ import org.apache.kylin.cube.cuboid.CuboidScheduler;
import org.apache.kylin.engine.mr.common.BatchConstants;
import org.apache.kylin.measure.BufferedMeasureCodec;
import org.apache.kylin.measure.hllc.HLLCounter;
+import org.apache.kylin.metadata.datatype.DataType;
import org.apache.kylin.metadata.model.TblColRef;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -40,6 +41,7 @@ import com.google.common.hash.HashFunction;
import com.google.common.hash.Hasher;
import com.google.common.hash.Hashing;
+
/**
*/
public class FactDistinctColumnsMapper<KEYIN> extends FactDistinctColumnsMapperBase<KEYIN, Object> {
@@ -50,6 +52,7 @@ public class FactDistinctColumnsMapper<KEYIN> extends FactDistinctColumnsMapperB
BYTES
}
+
protected boolean collectStatistics = false;
protected CuboidScheduler cuboidScheduler = null;
protected int nRowKey;
@@ -68,6 +71,8 @@ public class FactDistinctColumnsMapper<KEYIN> extends FactDistinctColumnsMapperB
private int partitionColumnIndex = -1;
private boolean needFetchPartitionCol = true;
+ private SelfDefineSortableKey sortableKey = new SelfDefineSortableKey();
+
@Override
protected void setup(Context context) throws IOException {
super.setup(context);
@@ -137,7 +142,6 @@ public class FactDistinctColumnsMapper<KEYIN> extends FactDistinctColumnsMapperB
String[] row = flatTableInputFormat.parseMapperInput(record);
context.getCounter(RawDataCounter.BYTES).increment(countSizeInBytes(row));
-
for (int i = 0; i < factDictCols.size(); i++) {
String fieldValue = row[dictionaryColumnIndex[i]];
if (fieldValue == null)
@@ -161,9 +165,9 @@ public class FactDistinctColumnsMapper<KEYIN> extends FactDistinctColumnsMapperB
tmpbuf.put(Bytes.toBytes(reducerIndex)[3]);
tmpbuf.put(valueBytes);
outputKey.set(tmpbuf.array(), 0, tmpbuf.position());
- sortableKey.setText(outputKey);
+ DataType type = factDictCols.get(i).getType();
+ sortableKey.init(outputKey, type);
//judge type
- sortableKey.setTypeIdByDatatype(factDictCols.get(i).getType());
context.write(sortableKey, EMPTY_TEXT);
// log a few rows for troubleshooting
@@ -189,8 +193,7 @@ public class FactDistinctColumnsMapper<KEYIN> extends FactDistinctColumnsMapperB
tmpbuf.put(MARK_FOR_PARTITION_COL);
tmpbuf.put(valueBytes);
outputKey.set(tmpbuf.array(), 0, tmpbuf.position());
- sortableKey.setText(outputKey);
- sortableKey.setTypeId((byte) 0);
+ sortableKey.init(outputKey, (byte) 0);
context.write(sortableKey, EMPTY_TEXT);
}
}
@@ -247,13 +250,13 @@ public class FactDistinctColumnsMapper<KEYIN> extends FactDistinctColumnsMapperB
hllBuf.clear();
hll.writeRegisters(hllBuf);
outputValue.set(hllBuf.array(), 0, hllBuf.position());
- sortableKey.setText(outputKey);
- sortableKey.setTypeId((byte) 0);
+ sortableKey.init(outputKey, (byte) 0);
context.write(sortableKey, outputValue);
}
}
}
+
private int countNewSize(int oldSize, int dataSize) {
int newSize = oldSize * 2;
while (newSize < dataSize) {
@@ -261,4 +264,5 @@ public class FactDistinctColumnsMapper<KEYIN> extends FactDistinctColumnsMapperB
}
return newSize;
}
+
}
http://git-wip-us.apache.org/repos/asf/kylin/blob/77fe5813/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapperBase.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapperBase.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapperBase.java
index 34624e3..458af69 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapperBase.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/FactDistinctColumnsMapperBase.java
@@ -54,7 +54,7 @@ abstract public class FactDistinctColumnsMapperBase<KEYIN, VALUEIN> extends Kyli
protected IMRTableInputFormat flatTableInputFormat;
protected Text outputKey = new Text();
- protected SelfDefineSortableKey sortableKey = new SelfDefineSortableKey();
+ //protected SelfDefineSortableKey sortableKey = new SelfDefineSortableKey();
protected Text outputValue = new Text();
protected int errorRecordCounter = 0;
http://git-wip-us.apache.org/repos/asf/kylin/blob/77fe5813/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/SelfDefineSortableKey.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/SelfDefineSortableKey.java b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/SelfDefineSortableKey.java
index 74525d3..c75abea 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/SelfDefineSortableKey.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/SelfDefineSortableKey.java
@@ -17,20 +17,16 @@
*/
package org.apache.kylin.engine.mr.steps;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.WritableComparable;
-import org.apache.kylin.metadata.datatype.DataType;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
import java.io.DataInput;
import java.io.DataOutput;
import java.io.IOException;
-/**
- * Created by xiefan on 16-11-1.
- */
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.kylin.metadata.datatype.DataType;
+
public class SelfDefineSortableKey implements WritableComparable<SelfDefineSortableKey> {
+
public enum TypeFlag {
NONE_NUMERIC_TYPE,
INTEGER_FAMILY_TYPE,
@@ -39,51 +35,48 @@ public class SelfDefineSortableKey implements WritableComparable<SelfDefineSorta
private byte typeId; //non-numeric(0000 0000) int(0000 0001) other numberic(0000 0010)
- private Text text;
+ private Text rawKey;
- private static final Logger logger = LoggerFactory.getLogger(SelfDefineSortableKey.class);
+ private Object keyInObj;
public SelfDefineSortableKey() {
}
- public SelfDefineSortableKey(byte typeId, Text text) {
+ public SelfDefineSortableKey(Text key, DataType type) {
+ init(key, getTypeIdByDatatype(type));
+ }
+
+ public void init(Text key, byte typeId) {
this.typeId = typeId;
- this.text = text;
+ this.rawKey = key;
+ if (isNumberFamily()) {
+ String valueStr = new String(key.getBytes(), 1, key.getLength() - 1);
+ if (isIntegerFamily()) {
+ this.keyInObj = Long.parseLong(valueStr);
+ } else {
+ this.keyInObj = Double.parseDouble(valueStr);
+ }
+ } else {
+ this.keyInObj = key;
+ }
+ }
+
+
+ public void init(Text key, DataType type) {
+ init(key, getTypeIdByDatatype(type));
}
@Override
public int compareTo(SelfDefineSortableKey o) {
- if (!o.isNumberFamily()) {
- return this.text.compareTo(o.text);
+ if (this.typeId != o.typeId)
+ throw new IllegalStateException("Error. Incompatible types");
+ if (!isNumberFamily()) {
+ return ((Text) this.keyInObj).compareTo(((Text) o.keyInObj));
} else {
- byte[] data1 = this.text.getBytes();
- byte[] data2 = o.text.getBytes();
- String str1 = new String(data1, 1, data1.length - 1);
- String str2 = new String(data2, 1, data2.length - 1);
- if (str1.equals("") || str2.equals("")) {
- //should not achieve here
- logger.error("none numeric value!");
- return 0;
- }
- if (o.isIntegerFamily()) { //integer type
- try {
- Long num1 = Long.parseLong(str1);
- Long num2 = Long.parseLong(str2);
- return num1.compareTo(num2);
- } catch (NumberFormatException e) {
- logger.error("NumberFormatException when parse integer family number.str1:" + str1 + " str2:" + str2);
- e.printStackTrace();
- return 0;
- }
- } else { //other numeric type
- try {
- Double num1 = Double.parseDouble(str1);
- Double num2 = Double.parseDouble(str2);
- return num1.compareTo(num2);
- } catch (NumberFormatException e) {
- logger.error("NumberFormatException when parse doul family number.str1:" + str1 + " str2:" + str2);
- return 0;
- }
+ if (isIntegerFamily()) {
+ return Long.compare((Long) this.keyInObj, (Long) o.keyInObj);
+ } else {
+ return Double.compare((Double) this.keyInObj, (Double) o.keyInObj);
}
}
}
@@ -91,14 +84,15 @@ public class SelfDefineSortableKey implements WritableComparable<SelfDefineSorta
@Override
public void write(DataOutput dataOutput) throws IOException {
dataOutput.writeByte(typeId);
- text.write(dataOutput);
+ rawKey.write(dataOutput);
}
@Override
public void readFields(DataInput dataInput) throws IOException {
this.typeId = dataInput.readByte();
- this.text = new Text();
- text.readFields(dataInput);
+ Text inputKey = new Text();
+ inputKey.readFields(dataInput);
+ init(inputKey, typeId);
}
public short getTypeId() {
@@ -106,7 +100,7 @@ public class SelfDefineSortableKey implements WritableComparable<SelfDefineSorta
}
public Text getText() {
- return text;
+ return rawKey;
}
public boolean isNumberFamily() {
@@ -119,25 +113,21 @@ public class SelfDefineSortableKey implements WritableComparable<SelfDefineSorta
return (typeId == TypeFlag.INTEGER_FAMILY_TYPE.ordinal());
}
- public boolean isOtherNumericFamily() {
- return (typeId == TypeFlag.DOUBLE_FAMILY_TYPE.ordinal());
- }
- public void setTypeIdByDatatype(DataType type) {
+ public byte getTypeIdByDatatype(DataType type) {
if (!type.isNumberFamily()) {
- this.typeId = (byte) TypeFlag.NONE_NUMERIC_TYPE.ordinal();
+ return (byte) TypeFlag.NONE_NUMERIC_TYPE.ordinal();
} else if (type.isIntegerFamily()) {
- this.typeId = (byte) TypeFlag.INTEGER_FAMILY_TYPE.ordinal();
+ return (byte) TypeFlag.INTEGER_FAMILY_TYPE.ordinal();
} else {
- this.typeId = (byte) TypeFlag.DOUBLE_FAMILY_TYPE.ordinal();
+ return (byte) TypeFlag.DOUBLE_FAMILY_TYPE.ordinal();
}
}
-
+
public void setTypeId(byte typeId) {
this.typeId = typeId;
}
- public void setText(Text text) {
- this.text = text;
- }
}
+
+
http://git-wip-us.apache.org/repos/asf/kylin/blob/77fe5813/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NumberDictionaryForestTest.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NumberDictionaryForestTest.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NumberDictionaryForestTest.java
index 1908992..c31377c 100644
--- a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NumberDictionaryForestTest.java
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NumberDictionaryForestTest.java
@@ -39,7 +39,6 @@ import org.apache.kylin.dict.NumberDictionary;
import org.apache.kylin.dict.NumberDictionaryBuilder;
import org.apache.kylin.dict.NumberDictionaryForestBuilder;
import org.apache.kylin.dict.TrieDictionaryForest;
-import org.apache.kylin.engine.mr.steps.SelfDefineSortableKey.TypeFlag;
import org.junit.Ignore;
import org.junit.Test;
@@ -49,19 +48,20 @@ import org.junit.Test;
public class NumberDictionaryForestTest {
+
@Test
public void testNumberDictionaryForestLong() {
List<String> list = randomLongData(100);
- testData(list, TypeFlag.INTEGER_FAMILY_TYPE);
+ testData(list, SelfDefineSortableKey.TypeFlag.INTEGER_FAMILY_TYPE);
}
@Test
public void testNumberDictionaryForestDouble() {
List<String> list = randomDoubleData(100);
- testData(list, TypeFlag.DOUBLE_FAMILY_TYPE);
+ testData(list, SelfDefineSortableKey.TypeFlag.DOUBLE_FAMILY_TYPE);
}
- private void testData(List<String> list, TypeFlag flag) {
+ private void testData(List<String> list, SelfDefineSortableKey.TypeFlag flag) {
//stimulate map-reduce job
ArrayList<SelfDefineSortableKey> keyList = createKeyList(list, (byte) flag.ordinal());
Collections.sort(keyList);
@@ -127,7 +127,7 @@ public class NumberDictionaryForestTest {
dict2.dump(System.out);
}
-
+
@Test
public void testMerge() {
// mimic the logic as in MergeCuboidMapper
@@ -136,7 +136,7 @@ public class NumberDictionaryForestTest {
b1.addValue("3");
b1.addValue("23");
TrieDictionaryForest<String> dict1 = b1.build();
-
+
NumberDictionaryForestBuilder b2 = new NumberDictionaryForestBuilder();
b2.addValue("0");
b2.addValue("2");
@@ -144,7 +144,7 @@ public class NumberDictionaryForestTest {
b2.addValue("15");
b2.addValue("23");
TrieDictionaryForest<String> dict2 = b2.build();
-
+
assertTrue(dict1.getSizeOfId() == dict2.getSizeOfId());
assertTrue(dict1.getSizeOfValue() == dict2.getSizeOfValue());
@@ -167,7 +167,7 @@ public class NumberDictionaryForestTest {
@Ignore
@Test
- public void testDecimalsWithBeginZero(){
+ public void testDecimalsWithBeginZero() {
List<String> testData = new ArrayList<>();
testData.add("000000000000000000000000000.4868");
testData.add("00000000000000000000000000000000000000");
@@ -235,7 +235,8 @@ public class NumberDictionaryForestTest {
//System.out.println("arrays toString:"+Arrays.toString(valueField));
Text outputKey = new Text();
outputKey.set(keyBuffer.array(), offset, keyBuffer.position() - offset);
- SelfDefineSortableKey sortableKey = new SelfDefineSortableKey(typeFlag, outputKey);
+ SelfDefineSortableKey sortableKey = new SelfDefineSortableKey();
+ sortableKey.init(outputKey, typeFlag);
keyList.add(sortableKey);
}
return keyList;
http://git-wip-us.apache.org/repos/asf/kylin/blob/77fe5813/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/SelfDefineSortableKeyTest.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/SelfDefineSortableKeyTest.java b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/SelfDefineSortableKeyTest.java
index df6287f..551998f 100644
--- a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/SelfDefineSortableKeyTest.java
+++ b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/SelfDefineSortableKeyTest.java
@@ -18,6 +18,7 @@
package org.apache.kylin.engine.mr.steps;
+import static junit.framework.TestCase.fail;
import static org.junit.Assert.assertTrue;
import java.nio.ByteBuffer;
@@ -29,8 +30,8 @@ import java.util.Random;
import java.util.UUID;
import org.apache.hadoop.io.Text;
+import org.apache.kylin.common.util.ByteArray;
import org.apache.kylin.common.util.Bytes;
-import org.apache.kylin.engine.mr.steps.SelfDefineSortableKey.TypeFlag;
import org.junit.Test;
/**
@@ -55,7 +56,7 @@ public class SelfDefineSortableKeyTest {
System.out.println("test numbers:" + longList);
ArrayList<String> strNumList = listToStringList(longList);
//System.out.println("test num strs list:"+strNumList);
- ArrayList<SelfDefineSortableKey> keyList = createKeyList(strNumList, (byte) TypeFlag.INTEGER_FAMILY_TYPE.ordinal());
+ ArrayList<SelfDefineSortableKey> keyList = createKeyList(strNumList, (byte) SelfDefineSortableKey.TypeFlag.INTEGER_FAMILY_TYPE.ordinal());
System.out.println(keyList.get(0).isIntegerFamily());
Collections.sort(keyList);
ArrayList<String> strListAftereSort = new ArrayList<>();
@@ -91,8 +92,7 @@ public class SelfDefineSortableKeyTest {
System.out.println("test numbers:" + doubleList);
ArrayList<String> strNumList = listToStringList(doubleList);
//System.out.println("test num strs list:"+strNumList);
- ArrayList<SelfDefineSortableKey> keyList = createKeyList(strNumList, (byte) TypeFlag.DOUBLE_FAMILY_TYPE.ordinal());
- System.out.println(keyList.get(0).isOtherNumericFamily());
+ ArrayList<SelfDefineSortableKey> keyList = createKeyList(strNumList, (byte) SelfDefineSortableKey.TypeFlag.DOUBLE_FAMILY_TYPE.ordinal());
Collections.sort(keyList);
ArrayList<String> strListAftereSort = new ArrayList<>();
for (SelfDefineSortableKey key : keyList) {
@@ -121,8 +121,7 @@ public class SelfDefineSortableKeyTest {
strList.add("hello"); //duplicate
strList.add("123");
strList.add("");
- ArrayList<SelfDefineSortableKey> keyList = createKeyList(strList, (byte) TypeFlag.NONE_NUMERIC_TYPE.ordinal());
- System.out.println(keyList.get(0).isOtherNumericFamily());
+ ArrayList<SelfDefineSortableKey> keyList = createKeyList(strList, (byte) SelfDefineSortableKey.TypeFlag.NONE_NUMERIC_TYPE.ordinal());
Collections.sort(keyList);
ArrayList<String> strListAftereSort = new ArrayList<>();
for (SelfDefineSortableKey key : keyList) {
@@ -138,6 +137,60 @@ public class SelfDefineSortableKeyTest {
}
@Test
+ public void testPerformance() {
+ Random rand = new Random(System.currentTimeMillis());
+ ArrayList<Double> doubleList = new ArrayList<>();
+ int count = 10 * 10000;
+ for (int i = 0; i < count; i++) {
+ doubleList.add(rand.nextDouble());
+ }
+ doubleList.add(0.0);
+ doubleList.add(0.0); //test duplicate
+ doubleList.add(-1.0); //test negative number
+ doubleList.add(Double.MAX_VALUE);
+ doubleList.add(-Double.MAX_VALUE);
+ //System.out.println(Double.MIN_VALUE);
+
+
+ ArrayList<String> strNumList = listToStringList(doubleList);
+ //System.out.println("test num strs list:"+strNumList);
+ ArrayList<SelfDefineSortableKey> keyList = createKeyList(strNumList, (byte) SelfDefineSortableKey.TypeFlag.DOUBLE_FAMILY_TYPE.ordinal());
+
+ System.out.println("start to test str sort");
+ long start = System.currentTimeMillis();
+ Collections.sort(strNumList);
+ System.out.println("sort time : " + (System.currentTimeMillis() - start));
+
+
+ System.out.println("start to test double sort");
+ start = System.currentTimeMillis();
+ Collections.sort(keyList);
+ System.out.println("sort time : " + (System.currentTimeMillis() - start));
+
+ List<ByteArray> byteList = new ArrayList<>();
+ for (String str : strNumList) {
+ byteList.add(new ByteArray(Bytes.toBytes(str)));
+ }
+ System.out.println("start to test byte array sort");
+ start = System.currentTimeMillis();
+ Collections.sort(byteList);
+ System.out.println("sort time : " + (System.currentTimeMillis() - start));
+
+ //test new key
+ List<SelfDefineSortableKey> newKeyList = new ArrayList<>();
+ for (String str : strNumList) {
+ SelfDefineSortableKey key = new SelfDefineSortableKey();
+ key.init(new Text(str), (byte) SelfDefineSortableKey.TypeFlag.DOUBLE_FAMILY_TYPE.ordinal());
+ newKeyList.add(key);
+ }
+ System.out.println("start to test new sortable key");
+ start = System.currentTimeMillis();
+ Collections.sort(newKeyList);
+ System.out.println("sort time : " + (System.currentTimeMillis() - start));
+ }
+
+
+ @Test
public void testIllegalNumber() {
Random rand = new Random(System.currentTimeMillis());
ArrayList<Double> doubleList = new ArrayList<>();
@@ -156,18 +209,19 @@ public class SelfDefineSortableKeyTest {
ArrayList<String> strNumList = listToStringList(doubleList);
strNumList.add("fjaeif"); //illegal type
//System.out.println("test num strs list:"+strNumList);
- ArrayList<SelfDefineSortableKey> keyList = createKeyList(strNumList, (byte) TypeFlag.DOUBLE_FAMILY_TYPE.ordinal());
- System.out.println(keyList.get(0).isOtherNumericFamily());
- Collections.sort(keyList);
- for (SelfDefineSortableKey key : keyList) {
- printKey(key);
+ try {
+ ArrayList<SelfDefineSortableKey> keyList = createKeyList(strNumList, (byte) SelfDefineSortableKey.TypeFlag.DOUBLE_FAMILY_TYPE.ordinal());
+ Collections.sort(keyList);
+ fail("Need catch exception");
+ }catch(Exception e){
+ //correct
}
}
@Test
public void testEnum() {
- TypeFlag flag = TypeFlag.DOUBLE_FAMILY_TYPE;
+ SelfDefineSortableKey.TypeFlag flag = SelfDefineSortableKey.TypeFlag.DOUBLE_FAMILY_TYPE;
System.out.println((byte) flag.ordinal());
int t = (byte) flag.ordinal();
System.out.println(t);
@@ -176,7 +230,6 @@ public class SelfDefineSortableKeyTest {
private <T> ArrayList<String> listToStringList(ArrayList<T> list) {
ArrayList<String> strList = new ArrayList<>();
for (T t : list) {
- System.out.println(t.toString());
strList.add(t.toString());
}
return strList;
@@ -193,7 +246,8 @@ public class SelfDefineSortableKeyTest {
Bytes.copy(keyBuffer.array(), 1, keyBuffer.position() - offset - 1);
Text outputKey = new Text();
outputKey.set(keyBuffer.array(), offset, keyBuffer.position() - offset);
- SelfDefineSortableKey sortableKey = new SelfDefineSortableKey(typeFlag, outputKey);
+ SelfDefineSortableKey sortableKey = new SelfDefineSortableKey();
+ sortableKey.init(outputKey, typeFlag);
keyList.add(sortableKey);
}
return keyList;