minor refactor of ByteArray

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

Branch: refs/heads/master
Commit: 38dae180038ab3bd1b9d80326c8c171ef9c36b07
Parents: 5d2f9cf
Author: Li Yang <liy...@apache.org>
Authored: Tue Sep 12 14:58:53 2017 +0800
Committer: Roger Shi <rogershijich...@gmail.com>
Committed: Fri Sep 15 15:09:25 2017 +0800

----------------------------------------------------------------------
 .../org/apache/kylin/common/util/ByteArray.java | 97 ++------------------
 .../cube/gridtable/ScanRangePlannerBase.java    | 10 +-
 .../org/apache/kylin/cube/util/CubingUtils.java | 12 +--
 .../kylin/gridtable/GTAggregateScanner.java     |  4 +-
 .../org/apache/kylin/gridtable/GTRecord.java    | 43 ++-------
 .../org/apache/kylin/gridtable/GTRowBlock.java  | 47 +---------
 .../gridtable/GTStreamAggregateScanner.java     | 15 +--
 .../gridtable/memstore/GTSimpleMemStore.java    |  3 +-
 .../kylin/measure/raw/RawMeasureType.java       |  2 +-
 .../kylin/measure/topn/TopNMeasureType.java     |  2 +-
 .../kylin/engine/mr/common/NDCuboidBuilder.java |  9 +-
 .../mr/steps/FactDistinctColumnsMapper.java     | 13 +--
 .../engine/mr/steps/MergeCuboidMapper.java      |  4 +-
 .../kylin/engine/mr/steps/CubeSamplingTest.java | 11 +--
 .../mr/steps/NewCubeSamplingMethodTest.java     | 35 +++----
 .../engine/spark/KylinKryoRegistrator.java      |  1 -
 .../apache/kylin/engine/spark/SparkCubing.java  | 12 +--
 17 files changed, 80 insertions(+), 240 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/38dae180/core-common/src/main/java/org/apache/kylin/common/util/ByteArray.java
----------------------------------------------------------------------
diff --git 
a/core-common/src/main/java/org/apache/kylin/common/util/ByteArray.java 
b/core-common/src/main/java/org/apache/kylin/common/util/ByteArray.java
index db9fc80..39d0e07 100644
--- a/core-common/src/main/java/org/apache/kylin/common/util/ByteArray.java
+++ b/core-common/src/main/java/org/apache/kylin/common/util/ByteArray.java
@@ -28,8 +28,6 @@ public class ByteArray implements Comparable<ByteArray>, 
Serializable {
 
     private static final long serialVersionUID = 1L;
 
-    public static final ByteArray EMPTY = new ImmutableByteArray();
-
     public static ByteArray allocate(int length) {
         return new ByteArray(new byte[length]);
     }
@@ -76,40 +74,6 @@ public class ByteArray implements Comparable<ByteArray>, 
Serializable {
         return length;
     }
 
-    public void set(byte[] array) {
-        set(array, 0, array.length);
-    }
-
-    public void set(byte[] array, int offset, int length) {
-        this.data = array;
-        this.offset = offset;
-        this.length = length;
-    }
-
-    public void set(ByteArray o) {
-        set(o.data, o.offset, o.length);
-    }
-
-    public void set(int offset, int length) {
-        this.offset = offset;
-        this.length = length;
-    }
-
-    public void setLength(int length) {
-        this.length = length;
-    }
-
-    public ByteArray copy() {
-        ByteArray copy;
-        if (data != null) {
-            copy = new ByteArray(length);
-        } else {
-            copy = new ByteArray(null);
-        }
-        copy.copyFrom(this);
-        return copy;
-    }
-
     //notice this will have a length header
     public void exportData(ByteBuffer out) {
         BytesUtil.writeByteArray(this.data, this.offset, this.length, out);
@@ -120,13 +84,6 @@ public class ByteArray implements Comparable<ByteArray>, 
Serializable {
         return new ByteArray(bytes);
     }
 
-    public void copyFrom(ByteArray other) {
-        if (other.data != null) {
-            System.arraycopy(other.array(), other.offset, data, offset, 
other.length);
-        }
-        this.length = other.length;
-    }
-
     public ByteBuffer asBuffer() {
         if (data == null)
             return null;
@@ -140,6 +97,16 @@ public class ByteArray implements Comparable<ByteArray>, 
Serializable {
         return Bytes.copy(this.array(), this.offset(), this.length());
     }
 
+    public void setLength(int pos) {
+        this.length = pos;
+    }
+
+    public void reset(byte[] data, int offset, int len) {
+        this.data = data;
+        this.offset = offset;
+        this.length = len;
+    }
+    
     @Override
     public int hashCode() {
         if (data == null) {
@@ -199,48 +166,4 @@ public class ByteArray implements Comparable<ByteArray>, 
Serializable {
             return Bytes.toStringBinary(data, offset, length);
     }
 
-    // 
============================================================================
-
-    public static class ImmutableByteArray extends ByteArray {
-
-        private static final long serialVersionUID = 1L;
-
-        public ImmutableByteArray() {
-            super();
-        }
-
-        public ImmutableByteArray(byte[] data, int offset, int length) {
-            super(data, offset, length);
-        }
-
-        public ImmutableByteArray(byte[] data) {
-            super(data);
-        }
-
-        @Override
-        public void set(byte[] array) {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public void set(byte[] array, int offset, int length) {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public void set(ByteArray o) {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public void setLength(int length) {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        public void copyFrom(ByteArray other) {
-            throw new UnsupportedOperationException();
-        }
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/38dae180/core-cube/src/main/java/org/apache/kylin/cube/gridtable/ScanRangePlannerBase.java
----------------------------------------------------------------------
diff --git 
a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/ScanRangePlannerBase.java
 
b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/ScanRangePlannerBase.java
index ed0a77a..ec86d42 100644
--- 
a/core-cube/src/main/java/org/apache/kylin/cube/gridtable/ScanRangePlannerBase.java
+++ 
b/core-cube/src/main/java/org/apache/kylin/cube/gridtable/ScanRangePlannerBase.java
@@ -42,6 +42,8 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
 public abstract class ScanRangePlannerBase {
+    
+    private static final ByteArray EMPTY = new ByteArray();
 
     //GT 
     protected GTInfo gtInfo;
@@ -161,8 +163,8 @@ public abstract class ScanRangePlannerBase {
 
     public class ColumnRange {
         public TblColRef column;
-        public ByteArray begin = ByteArray.EMPTY;
-        public ByteArray end = ByteArray.EMPTY;
+        public ByteArray begin = EMPTY;
+        public ByteArray end = EMPTY;
         public Set<ByteArray> valueSet;
         public boolean isBoundryInclusive;
 
@@ -208,8 +210,8 @@ public abstract class ScanRangePlannerBase {
 
         private void refreshBeginEndFromEquals() {
             if (valueSet.isEmpty()) {
-                begin = ByteArray.EMPTY;
-                end = ByteArray.EMPTY;
+                begin = EMPTY;
+                end = EMPTY;
             } else {
                 begin = rangeStartComparator.comparator.min(valueSet);
                 end = rangeEndComparator.comparator.max(valueSet);

http://git-wip-us.apache.org/repos/asf/kylin/blob/38dae180/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java
----------------------------------------------------------------------
diff --git 
a/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java 
b/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java
index ec4c04a..65719a5 100644
--- a/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java
+++ b/core-cube/src/main/java/org/apache/kylin/cube/util/CubingUtils.java
@@ -24,7 +24,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.Dictionary;
 import org.apache.kylin.cube.CubeInstance;
 import org.apache.kylin.cube.CubeSegment;
@@ -79,19 +78,16 @@ public class CubingUtils {
         }
 
         HashFunction hf = Hashing.murmur3_32();
-        ByteArray[] row_hashcodes = new ByteArray[rowkeyLength];
-        for (int i = 0; i < rowkeyLength; i++) {
-            row_hashcodes[i] = new ByteArray();
-        }
+        byte[][] row_hashcodes = new byte[rowkeyLength][];
         for (List<String> row : streams) {
             //generate hash for each row key column
             for (int i = 0; i < rowkeyLength; i++) {
                 Hasher hc = hf.newHasher();
                 final String cell = 
row.get(flatDesc.getRowKeyColumnIndexes()[i]);
                 if (cell != null) {
-                    row_hashcodes[i].set(hc.putString(cell).hash().asBytes());
+                    row_hashcodes[i] = hc.putString(cell).hash().asBytes();
                 } else {
-                    row_hashcodes[i].set(hc.putInt(0).hash().asBytes());
+                    row_hashcodes[i] = hc.putInt(0).hash().asBytes();
                 }
             }
 
@@ -101,7 +97,7 @@ public class CubingUtils {
                 Hasher hc = hf.newHasher();
                 final Integer[] cuboidBitSet = allCuboidsBitSet.get(cuboidId);
                 for (int position = 0; position < cuboidBitSet.length; 
position++) {
-                    hc.putBytes(row_hashcodes[cuboidBitSet[position]].array());
+                    hc.putBytes(row_hashcodes[cuboidBitSet[position]]);
                 }
                 counter.add(hc.hash().asBytes());
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/38dae180/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
----------------------------------------------------------------------
diff --git 
a/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java 
b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
index cbf4232..a3ad0f6 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTAggregateScanner.java
@@ -616,7 +616,7 @@ public class GTAggregateScanner implements IGTScanner, 
IGTBypassChecker {
                 for (int i = 0; i < dimensions.trueBitCount(); i++) {
                     int c = dimensions.trueBitAt(i);
                     final int columnLength = info.codeSystem.maxCodeLength(c);
-                    record.cols[c].set(key, offset, columnLength);
+                    record.cols[c].reset(key, offset, columnLength);
                     offset += columnLength;
                 }
 
@@ -629,7 +629,7 @@ public class GTAggregateScanner implements IGTScanner, 
IGTBypassChecker {
                 offset = 0;
                 for (int i = 0; i < value.length; i++) {
                     int col = metrics.trueBitAt(i);
-                    record.cols[col].set(bytes, offset, sizes[i]);
+                    record.cols[col].reset(bytes, offset, sizes[i]);
                     offset += sizes[i];
                 }
             }

http://git-wip-us.apache.org/repos/asf/kylin/blob/38dae180/core-cube/src/main/java/org/apache/kylin/gridtable/GTRecord.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTRecord.java 
b/core-cube/src/main/java/org/apache/kylin/gridtable/GTRecord.java
index 115fd91..ba3b1c4 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTRecord.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTRecord.java
@@ -27,7 +27,7 @@ import org.apache.kylin.common.util.ImmutableBitSet;
 
 import com.google.common.base.Preconditions;
 
-public class GTRecord implements Comparable<GTRecord>, Cloneable {
+public class GTRecord implements Comparable<GTRecord> {
 
     final transient GTInfo info;
     final ByteArray[] cols;
@@ -46,19 +46,10 @@ public class GTRecord implements Comparable<GTRecord>, 
Cloneable {
         this.info = info;
     }
 
-    @Override
-    public GTRecord clone() { // deep copy
-        ByteArray[] cols = new ByteArray[this.cols.length];
-        for (int i = 0; i < cols.length; i++) {
-            cols[i] = this.cols[i].copy();
-        }
-        return new GTRecord(this.info, cols);
-    }
-
     public void shallowCopyFrom(GTRecord source) {
         assert info == source.info;
         for (int i = 0; i < cols.length; i++) {
-            cols[i].set(source.cols[i]);
+            cols[i].reset(source.cols[i].array(), source.cols[i].offset(), 
source.cols[i].length());
         }
     }
 
@@ -75,7 +66,7 @@ public class GTRecord implements Comparable<GTRecord>, 
Cloneable {
     }
 
     public void set(int i, ByteArray data) {
-        cols[i].set(data.array(), data.offset(), data.length());
+        cols[i].reset(data.array(), data.offset(), data.length());
     }
 
     /** set record to the codes of specified values, new space allocated to 
hold the codes */
@@ -94,7 +85,7 @@ public class GTRecord implements Comparable<GTRecord>, 
Cloneable {
             int c = selectedCols.trueBitAt(i);
             info.codeSystem.encodeColumnValue(c, values[i], buf);
             int newPos = buf.position();
-            cols[c].set(buf.array(), buf.arrayOffset() + pos, newPos - pos);
+            cols[c].reset(buf.array(), buf.arrayOffset() + pos, newPos - pos);
             pos = newPos;
         }
         return this;
@@ -140,26 +131,6 @@ public class GTRecord implements Comparable<GTRecord>, 
Cloneable {
         return size;
     }
 
-    public GTRecord copy() {
-        return copy(info.colAll);
-    }
-
-    public GTRecord copy(ImmutableBitSet selectedCols) {
-        int len = sizeOf(selectedCols);
-        byte[] space = new byte[len];
-
-        GTRecord copy = new GTRecord(info);
-        int pos = 0;
-        for (int i = 0; i < selectedCols.trueBitCount(); i++) {
-            int c = selectedCols.trueBitAt(i);
-            System.arraycopy(cols[c].array(), cols[c].offset(), space, pos, 
cols[c].length());
-            copy.cols[c].set(space, pos, cols[c].length());
-            pos += cols[c].length();
-        }
-
-        return copy;
-    }
-
     @Override
     public boolean equals(Object obj) {
         if (this == obj)
@@ -289,7 +260,7 @@ public class GTRecord implements Comparable<GTRecord>, 
Cloneable {
         int pos = buf.position();
         for (int c : selectedCols) {
             int len = info.codeSystem.codeLength(c, buf);
-            cols[c].set(buf.array(), buf.arrayOffset() + pos, len);
+            cols[c].reset(buf.array(), buf.arrayOffset() + pos, len);
             pos += len;
             buf.position(pos);
         }
@@ -301,7 +272,7 @@ public class GTRecord implements Comparable<GTRecord>, 
Cloneable {
     public void loadColumns(int selectedCol, ByteBuffer buf) {
         int pos = buf.position();
         int len = info.codeSystem.codeLength(selectedCol, buf);
-        cols[selectedCol].set(buf.array(), buf.arrayOffset() + pos, len);
+        cols[selectedCol].reset(buf.array(), buf.arrayOffset() + pos, len);
     }
 
     public void loadColumnsFromColumnBlocks(ImmutableBitSet[] 
selectedColumnBlocks, ImmutableBitSet selectedCols,
@@ -312,7 +283,7 @@ public class GTRecord implements Comparable<GTRecord>, 
Cloneable {
                 int c = selectedColBlock.trueBitAt(i);
                 int len = info.codeSystem.codeLength(c, buf);
                 if (selectedCols.get(c)) {
-                    cols[c].set(buf.array(), buf.arrayOffset() + pos, len);
+                    cols[c].reset(buf.array(), buf.arrayOffset() + pos, len);
                 }
                 pos += len;
                 buf.position(pos);

http://git-wip-us.apache.org/repos/asf/kylin/blob/38dae180/core-cube/src/main/java/org/apache/kylin/gridtable/GTRowBlock.java
----------------------------------------------------------------------
diff --git a/core-cube/src/main/java/org/apache/kylin/gridtable/GTRowBlock.java 
b/core-cube/src/main/java/org/apache/kylin/gridtable/GTRowBlock.java
index 2cbbd8d..8380e0c 100644
--- a/core-cube/src/main/java/org/apache/kylin/gridtable/GTRowBlock.java
+++ b/core-cube/src/main/java/org/apache/kylin/gridtable/GTRowBlock.java
@@ -33,12 +33,12 @@ public class GTRowBlock {
         GTRowBlock b = new GTRowBlock(info);
 
         byte[] array = new byte[info.getMaxColumnLength(info.primaryKey)];
-        b.primaryKey.set(array);
+        b.primaryKey.reset(array, 0, array.length);
 
         int maxRows = info.isRowBlockEnabled() ? info.rowBlockSize : 1;
         for (int i = 0; i < b.cellBlocks.length; i++) {
             array = new byte[info.getMaxColumnLength(info.colBlocks[i]) * 
maxRows];
-            b.cellBlocks[i].set(array);
+            b.cellBlocks[i].reset(array, 0, array.length);
         }
         return b;
     }
@@ -86,18 +86,6 @@ public class GTRowBlock {
             }
         }
 
-        public void copyFrom(GTRowBlock other) {
-            assert info == other.info;
-
-            seqId = other.seqId;
-            nRows = other.nRows;
-            primaryKey.copyFrom(other.primaryKey);
-            for (int i = 0; i < info.colBlocks.length; i++) {
-                cellBlockBuffers[i].clear();
-                cellBlockBuffers[i].put(other.cellBlocks[i].array(), 
other.cellBlocks[i].offset(), other.cellBlocks[i].length());
-            }
-        }
-
         public void append(GTRecord r) {
             // add record to block
             if (isEmpty()) {
@@ -163,17 +151,6 @@ public class GTRowBlock {
         }
     }
 
-    public GTRowBlock copy() {
-        GTRowBlock copy = new GTRowBlock(info);
-
-        ByteBuffer buf = ByteBuffer.allocate(this.exportLength());
-        this.export(buf);
-        buf.clear();
-        copy.load(buf);
-
-        return copy;
-    }
-
     public boolean isEmpty() {
         return nRows == 0;
     }
@@ -253,25 +230,7 @@ public class GTRowBlock {
         byte[] data = result.array();
         int len = in.readInt();
         in.read(data, 0, len);
-        result.set(data, 0, len);
-    }
-
-    /** change pointers to point to data in given buffer, UNLIKE deserialize */
-    public void load(ByteBuffer buf) {
-        seqId = buf.getInt();
-        nRows = info.isRowBlockEnabled() ? buf.getInt() : 1;
-        load(primaryKey, buf);
-        for (int i = 0; i < info.colBlocks.length; i++) {
-            ByteArray cb = cellBlocks[i];
-            load(cb, buf);
-        }
-    }
-
-    private void load(ByteArray array, ByteBuffer buf) {
-        int len = buf.getInt();
-        int pos = buf.position();
-        array.set(buf.array(), buf.arrayOffset() + pos, len);
-        buf.position(pos + len);
+        result.setLength(len);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/38dae180/core-cube/src/main/java/org/apache/kylin/gridtable/GTStreamAggregateScanner.java
----------------------------------------------------------------------
diff --git 
a/core-cube/src/main/java/org/apache/kylin/gridtable/GTStreamAggregateScanner.java
 
b/core-cube/src/main/java/org/apache/kylin/gridtable/GTStreamAggregateScanner.java
index 61163d3..2a3e4b8 100644
--- 
a/core-cube/src/main/java/org/apache/kylin/gridtable/GTStreamAggregateScanner.java
+++ 
b/core-cube/src/main/java/org/apache/kylin/gridtable/GTStreamAggregateScanner.java
@@ -18,16 +18,17 @@
 
 package org.apache.kylin.gridtable;
 
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.PeekingIterator;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
 import org.apache.kylin.common.util.ImmutableBitSet;
 import org.apache.kylin.measure.BufferedMeasureCodec;
 import org.apache.kylin.measure.MeasureAggregator;
 
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.PeekingIterator;
 
 /**
  * GTStreamAggregateScanner requires input records to be sorted on group 
fields.
@@ -161,7 +162,7 @@ public class GTStreamAggregateScanner extends 
GTForwardingScanner {
             int offset = 0;
             for (int i = 0; i < metrics.trueBitCount(); i++) {
                 int c = metrics.trueBitAt(i);
-                returnRecord.cols[c].set(bytes, offset, sizes[i]);
+                returnRecord.cols[c].reset(bytes, offset, sizes[i]);
                 offset += sizes[i];
             }
             return returnRecord;

http://git-wip-us.apache.org/repos/asf/kylin/blob/38dae180/core-cube/src/main/java/org/apache/kylin/gridtable/memstore/GTSimpleMemStore.java
----------------------------------------------------------------------
diff --git 
a/core-cube/src/main/java/org/apache/kylin/gridtable/memstore/GTSimpleMemStore.java
 
b/core-cube/src/main/java/org/apache/kylin/gridtable/memstore/GTSimpleMemStore.java
index e1b5406..27debec 100644
--- 
a/core-cube/src/main/java/org/apache/kylin/gridtable/memstore/GTSimpleMemStore.java
+++ 
b/core-cube/src/main/java/org/apache/kylin/gridtable/memstore/GTSimpleMemStore.java
@@ -79,7 +79,7 @@ public class GTSimpleMemStore implements IGTStore {
     private class Writer implements IGTWriter {
         @Override
         public void write(GTRecord r) throws IOException {
-            ByteArray byteArray = r.exportColumns(info.getAllColumns()).copy();
+            ByteArray byteArray = r.exportColumns(info.getAllColumns());
             assert byteArray.offset() == 0;
             assert byteArray.array().length == byteArray.length();
             rowList.add(byteArray.array());
@@ -98,6 +98,7 @@ public class GTSimpleMemStore implements IGTStore {
     public IGTScanner scan(GTScanRequest scanRequest) {
 
         return new IGTScanner() {
+            @SuppressWarnings("unused")
             long count;
 
             @Override

http://git-wip-us.apache.org/repos/asf/kylin/blob/38dae180/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
----------------------------------------------------------------------
diff --git 
a/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java 
b/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
index eee8fa6..2add060 100644
--- 
a/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
+++ 
b/core-metadata/src/main/java/org/apache/kylin/measure/raw/RawMeasureType.java
@@ -148,7 +148,7 @@ public class RawMeasureType extends 
MeasureType<List<ByteArray>> {
                         newId = mergedDict.getIdFromValue(v);
                     }
                     BytesUtil.writeUnsigned(newId, newIdBuf, bufOffset, 
mergedDict.getSizeOfId());
-                    c.set(newIdBuf, bufOffset, mergedDict.getSizeOfId());
+                    c.reset(newIdBuf, bufOffset, mergedDict.getSizeOfId());
                     bufOffset += mergedDict.getSizeOfId();
                 }
                 return value;

http://git-wip-us.apache.org/repos/asf/kylin/blob/38dae180/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 c33ca0f..9a04bdc 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
@@ -203,7 +203,7 @@ public class TopNMeasureType extends 
MeasureType<TopNCounter<ByteArray>> {
                         offset += dimensionEncodings[i].getLengthOfEncoding();
                     }
 
-                    c.getItem().set(newIdBuf, bufOffset, newKeyLength);
+                    c.getItem().reset(newIdBuf, bufOffset, newKeyLength);
                     bufOffset += newKeyLength;
                 }
                 return topNCounter;

http://git-wip-us.apache.org/repos/asf/kylin/blob/38dae180/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
index 9ab42ea..f1b40de 100644
--- 
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
@@ -18,6 +18,8 @@
 
 package org.apache.kylin.engine.mr.common;
 
+import java.io.Serializable;
+
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.Pair;
 import org.apache.kylin.common.util.SplittedBytes;
@@ -30,10 +32,9 @@ import org.apache.kylin.cube.kv.RowKeyEncoderProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.Serializable;
-
 /**
  */
+@SuppressWarnings("serial")
 public class NDCuboidBuilder implements Serializable {
 
     protected static final Logger logger = 
LoggerFactory.getLogger(NDCuboidBuilder.class);
@@ -84,9 +85,9 @@ public class NDCuboidBuilder implements Serializable {
 
         int fullKeySize = rowkeyEncoder.getBytesLength();
         while (newKeyBuf.array().length < fullKeySize) {
-            newKeyBuf.set(new byte[newKeyBuf.length() * 2]);
+            newKeyBuf = new ByteArray(newKeyBuf.length() * 2);
         }
-        newKeyBuf.set(0, fullKeySize);
+        newKeyBuf.setLength(fullKeySize);
 
         rowkeyEncoder.encode(new ByteArray(newKeyBodyBuf, 0, offset), 
newKeyBuf);
 

http://git-wip-us.apache.org/repos/asf/kylin/blob/38dae180/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 82bfbd6..2965e2f 100755
--- 
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
@@ -25,7 +25,6 @@ import java.util.List;
 
 import org.apache.hadoop.io.Text;
 import org.apache.kylin.common.KylinVersion;
-import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.common.util.StringUtil;
 import org.apache.kylin.cube.cuboid.CuboidScheduler;
@@ -67,7 +66,6 @@ public class FactDistinctColumnsMapper<KEYIN> extends 
FactDistinctColumnsMapperB
     private int samplingPercentage;
     //private ByteArray[] row_hashcodes = null;
     private long[] rowHashCodesLong = null;
-    private ByteArray[] row_hashcodes = null;
     private ByteBuffer tmpbuf;
     private static final Text EMPTY_TEXT = new Text();
     public static final byte MARK_FOR_PARTITION_COL = (byte) 0xFE;
@@ -119,10 +117,6 @@ public class FactDistinctColumnsMapper<KEYIN> extends 
FactDistinctColumnsMapperB
             //for KYLIN-2518 backward compatibility
             if (KylinVersion.isBefore200(cubeDesc.getVersion())) {
                 isUsePutRowKeyToHllNewAlgorithm = false;
-                row_hashcodes = new ByteArray[nRowKey];
-                for (int i = 0; i < nRowKey; i++) {
-                    row_hashcodes[i] = new ByteArray();
-                }
                 hf = Hashing.murmur3_32();
                 logger.info("Found KylinVersion : {}. Use old algorithm for 
cuboid sampling.", cubeDesc.getVersion());
             } else {
@@ -237,13 +231,14 @@ public class FactDistinctColumnsMapper<KEYIN> extends 
FactDistinctColumnsMapperB
 
     private void putRowKeyToHLLOld(String[] row) {
         //generate hash for each row key column
+        byte[][] rowHashCodes = new byte[nRowKey][];
         for (int i = 0; i < nRowKey; i++) {
             Hasher hc = hf.newHasher();
             String colValue = 
row[intermediateTableDesc.getRowKeyColumnIndexes()[i]];
             if (colValue != null) {
-                row_hashcodes[i].set(hc.putString(colValue).hash().asBytes());
+                rowHashCodes[i] = hc.putString(colValue).hash().asBytes();
             } else {
-                row_hashcodes[i].set(hc.putInt(0).hash().asBytes());
+                rowHashCodes[i] = hc.putInt(0).hash().asBytes();
             }
         }
 
@@ -251,7 +246,7 @@ public class FactDistinctColumnsMapper<KEYIN> extends 
FactDistinctColumnsMapperB
         for (int i = 0, n = allCuboidsBitSet.length; i < n; i++) {
             Hasher hc = hf.newHasher();
             for (int position = 0; position < allCuboidsBitSet[i].length; 
position++) {
-                
hc.putBytes(row_hashcodes[allCuboidsBitSet[i][position]].array());
+                hc.putBytes(rowHashCodes[allCuboidsBitSet[i][position]]);
             }
 
             allCuboidsHLL[i].add(hc.hash().asBytes());

http://git-wip-us.apache.org/repos/asf/kylin/blob/38dae180/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
----------------------------------------------------------------------
diff --git 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
index 6cd7c28..5a72faf 100755
--- 
a/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
+++ 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/steps/MergeCuboidMapper.java
@@ -209,9 +209,9 @@ public class MergeCuboidMapper extends KylinMapper<Text, 
Text, Text, Text> {
 
         int fullKeySize = rowkeyEncoder.getBytesLength();
         while (newKeyBuf.array().length < fullKeySize) {
-            newKeyBuf.set(new byte[newKeyBuf.length() * 2]);
+            newKeyBuf = new ByteArray(newKeyBuf.length() * 2);
         }
-        newKeyBuf.set(0, fullKeySize);
+        newKeyBuf.setLength(fullKeySize);
 
         rowkeyEncoder.encode(new ByteArray(newKeyBodyBuf, 0, bufOffset), 
newKeyBuf);
         outputKey.set(newKeyBuf.array(), 0, fullKeySize);

http://git-wip-us.apache.org/repos/asf/kylin/blob/38dae180/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeSamplingTest.java
----------------------------------------------------------------------
diff --git 
a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeSamplingTest.java
 
b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeSamplingTest.java
index 0c8c13b..916f9fa 100644
--- 
a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeSamplingTest.java
+++ 
b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/CubeSamplingTest.java
@@ -22,7 +22,6 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.commons.lang.RandomStringUtils;
-import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.Bytes;
 import org.apache.kylin.measure.hllc.HLLCounter;
 import org.junit.Before;
@@ -40,7 +39,6 @@ public class CubeSamplingTest {
     private static final int ROW_LENGTH = 10;
 
     private final List<String> row = new ArrayList<String>(ROW_LENGTH);
-    private final ByteArray[] row_index = new ByteArray[ROW_LENGTH];
 
     private Integer[][] allCuboidsBitSet;
     private HashFunction hf = null;
@@ -69,10 +67,6 @@ public class CubeSamplingTest {
         //  hf = Hashing.goodFastHash(32);
         //        hf = Hashing.md5();
         hf = Hashing.murmur3_32();
-
-        for (int i = 0; i < ROW_LENGTH; i++) {
-            row_index[i] = new ByteArray();
-        }
     }
 
     private void addCuboidBitSet(long cuboidId, List<Integer[]> 
allCuboidsBitSet) {
@@ -107,16 +101,17 @@ public class CubeSamplingTest {
     }
 
     private void putRowKeyToHLL(List<String> row) {
+        byte[][] row_index = new byte[ROW_LENGTH][];
         int x = 0;
         for (String field : row) {
             Hasher hc = hf.newHasher();
-            row_index[x++].set(hc.putString(field).hash().asBytes());
+            row_index[x++] = hc.putString(field).hash().asBytes();
         }
 
         for (int i = 0, n = allCuboidsBitSet.length; i < n; i++) {
             Hasher hc = hf.newHasher();
             for (int position = 0; position < allCuboidsBitSet[i].length; 
position++) {
-                hc.putBytes(row_index[allCuboidsBitSet[i][position]].array());
+                hc.putBytes(row_index[allCuboidsBitSet[i][position]]);
                 hc.putBytes(seperator);
             }
             allCuboidsHLL[i].add(hc.hash().asBytes());

http://git-wip-us.apache.org/repos/asf/kylin/blob/38dae180/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NewCubeSamplingMethodTest.java
----------------------------------------------------------------------
diff --git 
a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NewCubeSamplingMethodTest.java
 
b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NewCubeSamplingMethodTest.java
index f018f28..97e356d 100644
--- 
a/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NewCubeSamplingMethodTest.java
+++ 
b/engine-mr/src/test/java/org/apache/kylin/engine/mr/steps/NewCubeSamplingMethodTest.java
@@ -19,10 +19,12 @@
 
 package org.apache.kylin.engine.mr.steps;
 
-import com.google.common.collect.Lists;
-import com.google.common.hash.HashFunction;
-import com.google.common.hash.Hasher;
-import com.google.common.hash.Hashing;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.kylin.common.util.ByteArray;
 import org.apache.kylin.common.util.Bytes;
@@ -32,11 +34,10 @@ import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Random;
-import java.util.Set;
+import com.google.common.collect.Lists;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
 
 @Ignore
 public class NewCubeSamplingMethodTest {
@@ -76,7 +77,7 @@ public class NewCubeSamplingMethodTest {
 
     public void comparePerformanceBasic(final List<List<String>> rows) throws 
Exception {
         //old hash method
-        ByteArray[] colHashValues = getNewColHashValues(ROW_LENGTH);
+        byte[][] colHashValues = new byte[ROW_LENGTH][];
         HLLCounter[] cuboidCounters = 
getNewCuboidCounters(allCuboidsBitSet.length);
         long start = System.currentTimeMillis();
         for (List<String> row : rows) {
@@ -85,7 +86,7 @@ public class NewCubeSamplingMethodTest {
         long totalTime = System.currentTimeMillis() - start;
         System.out.println("old method cost time : " + totalTime);
         //new hash method
-        colHashValues = getNewColHashValues(ROW_LENGTH);
+        colHashValues = new byte[ROW_LENGTH][];
         cuboidCounters = getNewCuboidCounters(allCuboidsBitSet.length);
         start = System.currentTimeMillis();
         long[] valueHashLong = new long[allCuboidsBitSet.length];
@@ -105,19 +106,19 @@ public class NewCubeSamplingMethodTest {
             @Override
             public void run() throws Exception {
                 HLLCounter counter = new HLLCounter(14, RegisterType.DENSE);
-                final ByteArray[] colHashValues = 
getNewColHashValues(ROW_LENGTH);
+                final byte[][] colHashValues = new byte[ROW_LENGTH][];
                 HashFunction hf = Hashing.murmur3_32();
                 for (List<String> row : rows) {
 
                     int x = 0;
                     for (String field : row) {
                         Hasher hc = hf.newHasher();
-                        
colHashValues[x++].set(hc.putString(field).hash().asBytes());
+                        colHashValues[x++] = 
hc.putString(field).hash().asBytes();
                     }
 
                     Hasher hc = hf.newHasher();
                     for (int position = 0; position < colHashValues.length; 
position++) {
-                        hc.putBytes(colHashValues[position].array());
+                        hc.putBytes(colHashValues[position]);
                     }
                     counter.add(hc.hash().asBytes());
                 }
@@ -208,17 +209,17 @@ public class NewCubeSamplingMethodTest {
         void run() throws Exception;
     }
 
-    private void putRowKeyToHLL(List<String> row, ByteArray[] colHashValues, 
HLLCounter[] cuboidCounters, HashFunction hashFunction) {
+    private void putRowKeyToHLL(List<String> row, byte[][] colHashValues, 
HLLCounter[] cuboidCounters, HashFunction hashFunction) {
         int x = 0;
         for (String field : row) {
             Hasher hc = hashFunction.newHasher();
-            colHashValues[x++].set(hc.putString(field).hash().asBytes());
+            colHashValues[x++] = hc.putString(field).hash().asBytes();
         }
 
         for (int i = 0, n = allCuboidsBitSet.length; i < n; i++) {
             Hasher hc = hashFunction.newHasher();
             for (int position = 0; position < allCuboidsBitSet[i].length; 
position++) {
-                
hc.putBytes(colHashValues[allCuboidsBitSet[i][position]].array());
+                hc.putBytes(colHashValues[allCuboidsBitSet[i][position]]);
                 //hc.putBytes(seperator);
             }
             cuboidCounters[i].add(hc.hash().asBytes());

http://git-wip-us.apache.org/repos/asf/kylin/blob/38dae180/engine-spark/src/main/java/org/apache/kylin/engine/spark/KylinKryoRegistrator.java
----------------------------------------------------------------------
diff --git 
a/engine-spark/src/main/java/org/apache/kylin/engine/spark/KylinKryoRegistrator.java
 
b/engine-spark/src/main/java/org/apache/kylin/engine/spark/KylinKryoRegistrator.java
index e287739..9e8c6d8 100644
--- 
a/engine-spark/src/main/java/org/apache/kylin/engine/spark/KylinKryoRegistrator.java
+++ 
b/engine-spark/src/main/java/org/apache/kylin/engine/spark/KylinKryoRegistrator.java
@@ -120,7 +120,6 @@ public class KylinKryoRegistrator implements 
KryoRegistrator {
         
kyroClasses.add(org.apache.kylin.common.persistence.RootPersistentEntity.class);
         kyroClasses.add(org.apache.kylin.common.util.Array.class);
         kyroClasses.add(org.apache.kylin.common.util.ByteArray.class);
-        
kyroClasses.add(org.apache.kylin.common.util.ByteArray.ImmutableByteArray.class);
         kyroClasses.add(org.apache.kylin.common.util.Dictionary.class);
         kyroClasses.add(org.apache.kylin.common.util.OrderedProperties.class);
         kyroClasses.add(org.apache.kylin.common.util.Pair.class);

http://git-wip-us.apache.org/repos/asf/kylin/blob/38dae180/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 7205e1b..c8aee5d 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
@@ -50,7 +50,6 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.util.ToolRunner;
 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.Dictionary;
 import org.apache.kylin.common.util.OptionsHelper;
@@ -266,7 +265,7 @@ public class SparkCubing extends AbstractApplication {
         final int nRowKey = cubeDesc.getRowkey().getRowKeyColumns().length;
         final long baseCuboidId = Cuboid.getBaseCuboidId(cubeDesc);
         final Map<Long, Integer[]> allCuboidsBitSet = 
Maps.newHashMapWithExpectedSize(allCuboidIds.size());
-        final ByteArray[] row_hashcodes = new ByteArray[nRowKey];
+        final byte[][] row_hashcodes = new byte[nRowKey][];
 
         for (Long cuboidId : allCuboidIds) {
             Integer[] cuboidBitSet = new Integer[Long.bitCount(cuboidId)];
@@ -282,9 +281,6 @@ public class SparkCubing extends AbstractApplication {
             }
             allCuboidsBitSet.put(cuboidId, cuboidBitSet);
         }
-        for (int i = 0; i < nRowKey; ++i) {
-            row_hashcodes[i] = new ByteArray();
-        }
 
         final HashMap<Long, HLLCounter> samplingResult = 
rowJavaRDD.aggregate(zeroValue, new Function2<HashMap<Long, HLLCounter>, 
List<String>, HashMap<Long, HLLCounter>>() {
 
@@ -296,9 +292,9 @@ public class SparkCubing extends AbstractApplication {
                     Hasher hc = hashFunction.newHasher();
                     String colValue = v2.get(rowKeyColumnIndexes[i]);
                     if (colValue != null) {
-                        
row_hashcodes[i].set(hc.putString(colValue).hash().asBytes());
+                        row_hashcodes[i] = 
hc.putString(colValue).hash().asBytes();
                     } else {
-                        row_hashcodes[i].set(hc.putInt(0).hash().asBytes());
+                        row_hashcodes[i] = hc.putInt(0).hash().asBytes();
                     }
                 }
 
@@ -307,7 +303,7 @@ public class SparkCubing extends AbstractApplication {
                     HLLCounter counter = v1.get(entry.getKey());
                     final Integer[] cuboidBitSet = entry.getValue();
                     for (int position = 0; position < cuboidBitSet.length; 
position++) {
-                        
hc.putBytes(row_hashcodes[cuboidBitSet[position]].array());
+                        hc.putBytes(row_hashcodes[cuboidBitSet[position]]);
                     }
                     counter.add(hc.hash().asBytes());
                 }

Reply via email to