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/01/17 03:32:10 UTC

[05/21] kylin git commit: KYLIN-2386 Revert KYLIN-2349 and KYLIN-2353

KYLIN-2386 Revert KYLIN-2349 and KYLIN-2353


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

Branch: refs/heads/master-cdh5.7
Commit: 4b977215186281908a8c29741128242146a2b934
Parents: 362acd9
Author: gaodayue <ga...@meituan.com>
Authored: Fri Jan 13 16:01:00 2017 +0800
Committer: gaodayue <ga...@meituan.com>
Committed: Fri Jan 13 19:20:46 2017 +0800

----------------------------------------------------------------------
 .../kylin/measure/bitmap/BitmapCounter.java     | 144 ++++++-------------
 .../bitmap/BitmapDistinctCountAggFunc.java      |   2 +-
 .../kylin/measure/bitmap/BitmapSerializer.java  |  15 +-
 3 files changed, 60 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/4b977215/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 19d7f5d..ac932ce 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
@@ -34,9 +34,6 @@ import org.roaringbitmap.buffer.MutableRoaringBitmap;
 public class BitmapCounter implements Comparable<BitmapCounter>, java.io.Serializable {
 
     private MutableRoaringBitmap bitmap = new MutableRoaringBitmap();
-    private final int VERSION = 2;
-    private Integer count;
-    private ByteBuffer buffer;
 
     public BitmapCounter() {
     }
@@ -45,43 +42,18 @@ public class BitmapCounter implements Comparable<BitmapCounter>, java.io.Seriali
         merge(another);
     }
 
-    private MutableRoaringBitmap getBitmap() {
-        if (!bitmap.isEmpty()) {
-            return bitmap;
-        }
-
-        if (buffer != null) {
-            @SuppressWarnings("unused")
-            int version = buffer.getInt();
-            @SuppressWarnings("unused")
-            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() {
-        getBitmap().clear();
+        bitmap.clear();
     }
 
     public BitmapCounter clone() {
         BitmapCounter newCounter = new BitmapCounter();
-        newCounter.bitmap = getBitmap().clone();
+        newCounter.bitmap = bitmap.clone();
         return newCounter;
     }
 
     public void add(int value) {
-        getBitmap().add(value);
-        count = null;
+        bitmap.add(value);
     }
 
     public void add(String value) {
@@ -92,107 +64,68 @@ public class BitmapCounter implements Comparable<BitmapCounter>, java.io.Seriali
     }
 
     public void merge(BitmapCounter another) {
-        getBitmap().or(another.getBitmap());
-        count = null;
+        this.bitmap.or(another.bitmap);
     }
 
     public void intersect(BitmapCounter another) {
-        getBitmap().and(another.getBitmap());
-        count = null;
+        this.bitmap.and(another.bitmap);
     }
 
-    public int getCount() {
-        if (count != null) {
-            return count;
-        }
-
-        return getBitmap().getCardinality();
+    public long getCount() {
+        return this.bitmap.getCardinality();
     }
 
     public int getMemBytes() {
-        return getBitmap().getSizeInBytes();
+        return this.bitmap.getSizeInBytes();
     }
 
     public Iterator<Integer> iterator() {
-        return getBitmap().iterator();
+        return bitmap.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 + 4);
-        out.putInt(getCount());
         out.put(bb);
     }
 
     public void readRegisters(ByteBuffer in) throws IOException {
-        int mark = in.position();
-        int version = in.getInt();
-
-        // keep forward compatibility
-        if (version == VERSION) {
-            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) {
-        int mark = in.position();
-        int len;
-        int version = in.getInt();
-
-        // keep forward compatibility
-        if (version == VERSION) {
-            len = in.getInt();
+    @Override
+    public String toString() {
+        long count = getCount();
+        if (count <= 10) {
+            return "(" + count + ")" + bitmap.toString();
         } else {
-            in.position(mark);
-            try (DataInputStream is = new DataInputStream(new ByteBufferBackedInputStream(in))) {
-                MutableRoaringBitmap bitmap = new MutableRoaringBitmap();
-                bitmap.deserialize(is);
-                len = in.position() - mark;
-            } catch (IOException e) {
-                throw new IllegalStateException(e);
+            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();
         }
-
-        in.position(mark);
-        return len;
     }
 
     @Override
     public int hashCode() {
         final int prime = 31;
         int result = 1;
-        result = prime * result + getBitmap().hashCode();
+        result = prime * result + bitmap.hashCode();
         return result;
     }
 
@@ -205,7 +138,7 @@ public class BitmapCounter implements Comparable<BitmapCounter>, java.io.Seriali
         if (getClass() != obj.getClass())
             return false;
         BitmapCounter other = (BitmapCounter) obj;
-        return getBitmap().equals(other.getBitmap());
+        return bitmap.equals(other.bitmap);
     }
 
     @Override
@@ -223,5 +156,20 @@ public class BitmapCounter implements Comparable<BitmapCounter>, java.io.Seriali
         else
             return -1;
     }
-    
+
+    public int peekLength(ByteBuffer in) {
+        int mark = in.position();
+        int len;
+
+        MutableRoaringBitmap bitmap = new MutableRoaringBitmap();
+        try (DataInputStream is = new DataInputStream(new ByteBufferBackedInputStream(in))) {
+            bitmap.deserialize(is);
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+
+        len = in.position() - mark;
+        in.position(mark);
+        return len;
+    }
 }

http://git-wip-us.apache.org/repos/asf/kylin/blob/4b977215/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 3a1a800..d039b6d 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 c;
+            return new BitmapCounter(c);
         } else {
             counter.merge(c);
             return counter;

http://git-wip-us.apache.org/repos/asf/kylin/blob/4b977215/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 4890295..089d18c 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,6 +29,8 @@ import org.apache.kylin.metadata.datatype.DataTypeSerializer;
  */
 public class BitmapSerializer extends DataTypeSerializer<BitmapCounter> {
 
+    private ThreadLocal<BitmapCounter> current = new ThreadLocal<>();
+
     public BitmapSerializer(DataType type) {
     }
 
@@ -41,9 +43,18 @@ 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 = new BitmapCounter();
+        BitmapCounter counter = current();
         try {
             counter.readRegisters(in);
         } catch (IOException e) {
@@ -54,7 +65,7 @@ public class BitmapSerializer extends DataTypeSerializer<BitmapCounter> {
 
     @Override
     public int peekLength(ByteBuffer in) {
-        return new BitmapCounter().peekLength(in);
+        return current().peekLength(in);
     }
 
     @Override