KYLIN-2353 Serialize BitmapCounter with distinct count Signed-off-by: Yang Li <liy...@apache.org>
Project: http://git-wip-us.apache.org/repos/asf/kylin/repo Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/731a53a5 Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/731a53a5 Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/731a53a5 Branch: refs/heads/master-hbase1.x Commit: 731a53a51a32fb1314b4e1282f158e3b7ed819a5 Parents: aa57446 Author: kangkaisen <kangkai...@live.com> Authored: Sat Dec 31 20:28:30 2016 +0800 Committer: Yang Li <liy...@apache.org> Committed: Sat Jan 7 19:08:37 2017 +0800 ---------------------------------------------------------------------- .../kylin/measure/bitmap/BitmapCounter.java | 105 ++++++++++++------- .../bitmap/BitmapDistinctCountAggFunc.java | 2 +- .../kylin/measure/bitmap/BitmapSerializer.java | 15 +-- 3 files changed, 70 insertions(+), 52 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/kylin/blob/731a53a5/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 aeb14ba..a18ac4a 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 @@ -35,6 +35,8 @@ public class BitmapCounter implements Comparable<BitmapCounter> { private MutableRoaringBitmap bitmap = new MutableRoaringBitmap(); private final int VERSION = 2; + private Integer count; + private ByteBuffer buffer; public BitmapCounter() { } @@ -43,18 +45,41 @@ public class BitmapCounter implements Comparable<BitmapCounter> { merge(another); } + private MutableRoaringBitmap getBitmap() { + if (!bitmap.isEmpty()) { + return bitmap; + } + + if (buffer != null) { + int version = buffer.getInt(); + int size = buffer.getInt(); + count = buffer.getInt(); + + try (DataInputStream is = new DataInputStream(new ByteBufferBackedInputStream(buffer))) { + bitmap.deserialize(is); + } catch (IOException e) { + throw new RuntimeException("deserialize bitmap failed!"); + } + + buffer = null; + } + + return bitmap; + } + public void clear() { - bitmap.clear(); + getBitmap().clear(); } public BitmapCounter clone() { BitmapCounter newCounter = new BitmapCounter(); - newCounter.bitmap = bitmap.clone(); + newCounter.bitmap = getBitmap().clone(); return newCounter; } public void add(int value) { - bitmap.add(value); + getBitmap().add(value); + count = getBitmap().getCardinality(); } public void add(byte[] value) { @@ -77,34 +102,43 @@ public class BitmapCounter implements Comparable<BitmapCounter> { } public void merge(BitmapCounter another) { - this.bitmap.or(another.bitmap); + getBitmap().or(another.getBitmap()); + count = getBitmap().getCardinality(); } public void intersect(BitmapCounter another) { - this.bitmap.and(another.bitmap); + getBitmap().and(another.getBitmap()); + count = getBitmap().getCardinality(); } - public long getCount() { - return this.bitmap.getCardinality(); + public int getCount() { + if (count != null) { + return count; + } + + return getBitmap().getCardinality(); } public int getMemBytes() { - return this.bitmap.getSizeInBytes(); + return getBitmap().getSizeInBytes(); } public Iterator<Integer> iterator() { - return bitmap.iterator(); + return getBitmap().iterator(); } public void writeRegisters(ByteBuffer out) throws IOException { ByteArrayOutputStream bos = new ByteArrayOutputStream(); DataOutputStream dos = new DataOutputStream(bos); + MutableRoaringBitmap bitmap = getBitmap(); bitmap.runOptimize(); bitmap.serialize(dos); dos.close(); ByteBuffer bb = ByteBuffer.wrap(bos.toByteArray()); + out.putInt(VERSION); - out.putInt(bos.size() + 4 + 4); + out.putInt(bos.size() + 4 + 4 + 4); + out.putInt(getCount()); out.put(bb); } @@ -116,13 +150,30 @@ public class BitmapCounter implements Comparable<BitmapCounter> { if (version == VERSION) { @SuppressWarnings("unused") int size = in.getInt(); + count = in.getInt(); + in.position(mark); + buffer = cloneBuffer(in, size); } else { in.position(mark); + try (DataInputStream is = new DataInputStream(new ByteBufferBackedInputStream(in))) { + getBitmap().deserialize(is); + } } + } - try (DataInputStream is = new DataInputStream(new ByteBufferBackedInputStream(in))) { - bitmap.deserialize(is); - } + private ByteBuffer cloneBuffer(ByteBuffer src, int size) throws IOException { + int mark = src.position(); + int limit = src.limit(); + + src.limit(mark + size); + ByteBuffer clone = ByteBuffer.allocate(size); + clone.put(src.slice()); + clone.flip(); + + src.position(mark + size); + src.limit(limit); + + return clone; } public int peekLength(ByteBuffer in) { @@ -132,7 +183,7 @@ public class BitmapCounter implements Comparable<BitmapCounter> { // keep forward compatibility if (version == VERSION) { - len = in.getInt() ; + len = in.getInt(); } else { in.position(mark); try (DataInputStream is = new DataInputStream(new ByteBufferBackedInputStream(in))) { @@ -149,32 +200,10 @@ public class BitmapCounter implements Comparable<BitmapCounter> { } @Override - public String toString() { - long count = getCount(); - if (count <= 10) { - return "(" + count + ")" + bitmap.toString(); - } else { - StringBuilder sb = new StringBuilder(); - sb.append("(").append(count).append("){"); - int values = 0; - for (Integer v : bitmap) { - if (values++ < 10) { - sb.append(v).append(","); - } else { - sb.append("..."); - break; - } - } - sb.append("}"); - return sb.toString(); - } - } - - @Override public int hashCode() { final int prime = 31; int result = 1; - result = prime * result + bitmap.hashCode(); + result = prime * result + getBitmap().hashCode(); return result; } @@ -187,7 +216,7 @@ public class BitmapCounter implements Comparable<BitmapCounter> { if (getClass() != obj.getClass()) return false; BitmapCounter other = (BitmapCounter) obj; - return bitmap.equals(other.bitmap); + return getBitmap().equals(other.getBitmap()); } @Override http://git-wip-us.apache.org/repos/asf/kylin/blob/731a53a5/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapDistinctCountAggFunc.java ---------------------------------------------------------------------- diff --git a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapDistinctCountAggFunc.java b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapDistinctCountAggFunc.java index d039b6d..3a1a800 100644 --- a/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapDistinctCountAggFunc.java +++ b/core-metadata/src/main/java/org/apache/kylin/measure/bitmap/BitmapDistinctCountAggFunc.java @@ -35,7 +35,7 @@ public class BitmapDistinctCountAggFunc { public static BitmapCounter add(BitmapCounter counter, Object v) { BitmapCounter c = (BitmapCounter) v; if (counter == null) { - return new BitmapCounter(c); + return c; } else { counter.merge(c); return counter; http://git-wip-us.apache.org/repos/asf/kylin/blob/731a53a5/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..4890295 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,8 +29,6 @@ import org.apache.kylin.metadata.datatype.DataTypeSerializer; */ public class BitmapSerializer extends DataTypeSerializer<BitmapCounter> { - private ThreadLocal<BitmapCounter> current = new ThreadLocal<>(); - public BitmapSerializer(DataType type) { } @@ -43,18 +41,9 @@ public class BitmapSerializer extends DataTypeSerializer<BitmapCounter> { } } - private BitmapCounter current() { - BitmapCounter counter = current.get(); - if (counter == null) { - counter = new BitmapCounter(); - current.set(counter); - } - return counter; - } - @Override public BitmapCounter deserialize(ByteBuffer in) { - BitmapCounter counter = current(); + BitmapCounter counter = new BitmapCounter(); try { counter.readRegisters(in); } catch (IOException e) { @@ -65,7 +54,7 @@ public class BitmapSerializer extends DataTypeSerializer<BitmapCounter> { @Override public int peekLength(ByteBuffer in) { - return current().peekLength(in); + return new BitmapCounter().peekLength(in); } @Override