View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.util;
19  
20  import java.util.concurrent.atomic.AtomicBoolean;
21  import java.util.concurrent.atomic.AtomicLongFieldUpdater;
22  import java.util.concurrent.atomic.AtomicReference;
23  
24  import org.apache.hadoop.hbase.classification.InterfaceAudience;
25  import org.apache.hadoop.hbase.classification.InterfaceStability;
26  
27  /**
28   * High scalable counter. Thread safe.
29   */
30  @InterfaceAudience.Public
31  @InterfaceStability.Evolving
32  public class Counter {
33    private static final int MAX_CELLS_LENGTH = 1 << 20;
34  
35    private static class Cell {
36      // Pads are added around the value to avoid cache-line contention with
37      // another cell's value. The cache-line size is expected to be equal to or
38      // less than about 128 Bytes (= 64 Bits * 16).
39  
40      @SuppressWarnings("unused")
41      volatile long p0, p1, p2, p3, p4, p5, p6;
42      volatile long value;
43      @SuppressWarnings("unused")
44      volatile long q0, q1, q2, q3, q4, q5, q6;
45  
46      static final AtomicLongFieldUpdater<Cell> valueUpdater =
47          AtomicLongFieldUpdater.newUpdater(Cell.class, "value");
48  
49      Cell() {}
50  
51      Cell(long initValue) {
52        value = initValue;
53      }
54  
55      long get() {
56        return value;
57      }
58  
59      boolean add(long delta) {
60        long current = value;
61        return valueUpdater.compareAndSet(this, current, current + delta);
62      }
63    }
64  
65    private static class Container {
66      /** The length should be a power of 2. */
67      final Cell[] cells;
68  
69      /** True if a new extended container is going to replace this. */
70      final AtomicBoolean demoted = new AtomicBoolean();
71  
72      Container(Cell cell) {
73        this(new Cell[] { cell });
74      }
75  
76      /**
77       * @param cells the length should be a power of 2
78       */
79      Container(Cell[] cells) {
80        this.cells = cells;
81      }
82    }
83  
84    private final AtomicReference<Container> containerRef;
85  
86    public Counter() {
87      this(new Cell());
88    }
89  
90    public Counter(long initValue) {
91      this(new Cell(initValue));
92    }
93  
94    private Counter(Cell initCell) {
95      containerRef = new AtomicReference<Container>(new Container(initCell));
96    }
97  
98    private static int hash() {
99      // The logic is borrowed from high-scale-lib's ConcurrentAutoTable.
100 
101     int h = System.identityHashCode(Thread.currentThread());
102     // You would think that System.identityHashCode on the current thread
103     // would be a good hash fcn, but actually on SunOS 5.8 it is pretty lousy
104     // in the low bits.
105 
106     h ^= (h >>> 20) ^ (h >>> 12); // Bit spreader, borrowed from Doug Lea
107     h ^= (h >>>  7) ^ (h >>>  4);
108     return h;
109   }
110 
111   public void add(long delta) {
112     Container container = containerRef.get();
113     Cell[] cells = container.cells;
114     int mask = cells.length - 1;
115 
116     int baseIndex = hash();
117     if(cells[baseIndex & mask].add(delta)) {
118       return;
119     }
120 
121     int index = baseIndex + 1;
122     while(true) {
123       if(cells[index & mask].add(delta)) {
124         break;
125       }
126       index++;
127     }
128 
129     if(index - baseIndex >= cells.length &&
130         cells.length < MAX_CELLS_LENGTH &&
131         container.demoted.compareAndSet(false, true)) {
132 
133       if(containerRef.get() == container) {
134         Cell[] newCells = new Cell[cells.length * 2];
135         System.arraycopy(cells, 0, newCells, 0, cells.length);
136         for(int i = cells.length; i < newCells.length; i++) {
137           newCells[i] = new Cell();
138           // Fill all of the elements with instances. Creating a cell on demand
139           // and putting it into the array makes a concurrent problem about
140           // visibility or, in other words, happens-before relation, because
141           // each element of the array is not volatile so that you should
142           // establish the relation by some piggybacking.
143         }
144         containerRef.compareAndSet(container, new Container(newCells));
145       }
146     }
147   }
148 
149   public void increment() {
150     add(1);
151   }
152 
153   public void decrement() {
154     add(-1);
155   }
156 
157   public void set(long value) {
158     containerRef.set(new Container(new Cell(value)));
159   }
160 
161   public long get() {
162     long sum = 0;
163     for(Cell cell : containerRef.get().cells) {
164       sum += cell.get();
165     }
166     return sum;
167   }
168 
169   public void destroy() {
170   }
171 
172   @Override
173   public String toString() {
174     Cell[] cells = containerRef.get().cells;
175 
176     long min = Long.MAX_VALUE;
177     long max = Long.MIN_VALUE;
178     long sum = 0;
179 
180     for(Cell cell : cells) {
181       long value = cell.get();
182       sum += value;
183       if(min > value) { min = value; }
184       if(max < value) { max = value; }
185     }
186 
187     return new StringBuilder(100)
188     .append("[value=").append(sum)
189     .append(", cells=[length=").append(cells.length)
190     .append(", min=").append(min)
191     .append(", max=").append(max)
192     .append("]]").toString();
193   }
194 }