1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.util;
21
22 import com.google.common.util.concurrent.ThreadFactoryBuilder;
23 import java.util.Map;
24 import java.util.Set;
25 import java.util.concurrent.ConcurrentHashMap;
26 import java.util.concurrent.ExecutorService;
27 import java.util.concurrent.Executors;
28 import java.util.concurrent.Future;
29 import java.util.concurrent.atomic.AtomicReference;
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.hbase.HConstants;
32 import org.apache.hadoop.hbase.classification.InterfaceAudience;
33 import org.slf4j.Logger;
34 import org.slf4j.LoggerFactory;
35
36
37
38
39
40
41
42
43
44
45
46 @InterfaceAudience.Private
47 public class LossyCounting<T> {
48 private static final Logger LOG = LoggerFactory.getLogger(LossyCounting.class);
49 private final ExecutorService executor;
50 private long bucketSize;
51 private int currentTerm;
52 private Map<T, Integer> data;
53 private long totalDataCount;
54 private final String name;
55 private LossyCountingListener<T> listener;
56 private static AtomicReference<Future<?>> fut = new AtomicReference<>(null);
57
58 public interface LossyCountingListener<T> {
59 void sweep(T key);
60 }
61
62 LossyCounting(String name, double errorRate) {
63 this(name, errorRate, null);
64 }
65
66 public LossyCounting(String name, double errorRate, LossyCountingListener<T> listener) {
67 if (errorRate < 0.0 || errorRate > 1.0) {
68 throw new IllegalArgumentException(" Lossy Counting error rate should be within range [0,1]");
69 }
70 this.name = name;
71 this.bucketSize = (long) Math.ceil(1 / errorRate);
72 this.currentTerm = 1;
73 this.totalDataCount = 0;
74 this.data = new ConcurrentHashMap<>();
75 this.listener = listener;
76 calculateCurrentTerm();
77 executor = Executors.newSingleThreadExecutor(
78 new ThreadFactoryBuilder().setDaemon(true).setNameFormat("lossy-count-%d").build());
79 }
80
81 LossyCounting(String name, Configuration conf) {
82 this(name, conf, null);
83 }
84
85 public LossyCounting(String name, Configuration conf, LossyCountingListener<T> listener) {
86 this(name, conf.getDouble(HConstants.DEFAULT_LOSSY_COUNTING_ERROR_RATE, 0.02), listener);
87 }
88
89 private void addByOne(T key) {
90
91
92 Integer i = data.get(key);
93 if (i == null) {
94 i = currentTerm != 0 ? currentTerm - 1 : 0;
95 }
96 data.put(key, i + 1);
97
98 totalDataCount++;
99 calculateCurrentTerm();
100 }
101
102 public void add(T key) {
103 addByOne(key);
104 if(totalDataCount % bucketSize == 0) {
105
106
107 Future<?> future = fut.get();
108 if (future != null && !future.isDone()){
109 return;
110 }
111 future = executor.submit(new SweepRunnable());
112 fut.set(future);
113 }
114 }
115
116
117
118
119 public void sweep() {
120 for(Map.Entry<T, Integer> entry : data.entrySet()) {
121 if(entry.getValue() < currentTerm) {
122 T metric = entry.getKey();
123 data.remove(metric);
124 if (listener != null) {
125 listener.sweep(metric);
126 }
127 }
128 }
129 }
130
131
132
133
134 private void calculateCurrentTerm() {
135 this.currentTerm = (int) Math.ceil(1.0 * totalDataCount / (double) bucketSize);
136 }
137
138 public long getBucketSize(){
139 return bucketSize;
140 }
141
142 public long getDataSize() {
143 return data.size();
144 }
145
146 public boolean contains(T key) {
147 return data.containsKey(key);
148 }
149
150 public Set<T> getElements(){
151 return data.keySet();
152 }
153
154 public long getCurrentTerm() {
155 return currentTerm;
156 }
157
158 class SweepRunnable implements Runnable {
159 @Override public void run() {
160 if (LOG.isTraceEnabled()) {
161 LOG.trace("Starting sweep of lossyCounting-" + name);
162 }
163 try {
164 sweep();
165 } catch (Exception exception) {
166 LOG.debug("Error while sweeping lossyCounting-" + name, exception);
167 }
168 }
169 }
170
171 public Future<?> getSweepFuture() {
172 return fut.get();
173 }
174 }
175