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.classification.InterfaceAudience;
25  import org.apache.hadoop.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    private static final int SUFFERABLE_SPIN_COUNT = 2;
35  
36    private static class Cell {
37      // Pads are added around the value to avoid cache-line contention with
38      // another cell's value. The cache-line size is expected to be equal to or
39      // less than about 128 Bytes (= 64 Bits * 16).
40  
41      @SuppressWarnings("unused")
42      volatile long p0, p1, p2, p3, p4, p5, p6;
43      volatile long value;
44      @SuppressWarnings("unused")
45      volatile long q0, q1, q2, q3, q4, q5, q6;
46  
47      static final AtomicLongFieldUpdater<Cell> valueUpdater =
48          AtomicLongFieldUpdater.newUpdater(Cell.class, "value");
49  
50      Cell() {}
51  
52      Cell(long initValue) {
53        value = initValue;
54      }
55  
56      long get() {
57        return value;
58      }
59  
60      boolean addAndIsCongested(long delta) {
61        for(int i = 0; i < SUFFERABLE_SPIN_COUNT; i++) {
62          if(add(delta)) {
63            return false;
64          }
65        }
66  
67        while(! add(delta)) {}
68  
69        return true;
70      }
71  
72      boolean add(long delta) {
73        long current = value;
74        return valueUpdater.compareAndSet(this, current, current + delta);
75      }
76    }
77  
78    private static class Container {
79      /** The length should be a power of 2. */
80      final Cell[] cells;
81  
82      /** True if a new extended container is going to replace this. */
83      final AtomicBoolean demoted = new AtomicBoolean();
84  
85      Container(Cell cell) {
86        this(new Cell[] { cell });
87      }
88  
89      /**
90       * @param cells the length should be a power of 2
91       */
92      Container(Cell[] cells) {
93        this.cells = cells;
94      }
95    }
96  
97    private final AtomicReference<Container> containerRef;
98  
99    public Counter() {
100     this(new Cell());
101   }
102 
103   public Counter(long initValue) {
104     this(new Cell(initValue));
105   }
106 
107   private Counter(Cell initCell) {
108     containerRef = new AtomicReference<Container>(new Container(initCell));
109   }
110 
111   private static int hash() {
112     return (int) Thread.currentThread().getId();
113   }
114 
115   public void add(long delta) {
116     Container container = containerRef.get();
117     Cell[] cells = container.cells;
118     int index = hash() & (cells.length - 1);
119     Cell cell = cells[index];
120 
121     if(cell.addAndIsCongested(delta) && cells.length < MAX_CELLS_LENGTH &&
122         container.demoted.compareAndSet(false, true)) {
123 
124       if(containerRef.get() == container) {
125         Cell[] newCells = new Cell[cells.length * 2];
126         System.arraycopy(cells, 0, newCells, 0, cells.length);
127         for(int i = cells.length; i < newCells.length; i++) {
128           newCells[i] = new Cell();
129           // Fill all of the elements with instances. Creating a cell on demand
130           // and putting it into the array makes a concurrent problem about
131           // visibility or, in other words, happens-before relation, because
132           // each element of the array is not volatile so that you should
133           // establish the relation by some piggybacking.
134         }
135         containerRef.compareAndSet(container, new Container(newCells));
136       }
137     }
138   }
139 
140   public void increment() {
141     add(1);
142   }
143 
144   public void decrement() {
145     add(-1);
146   }
147 
148   public void set(long value) {
149     containerRef.set(new Container(new Cell(value)));
150   }
151 
152   public long get() {
153     long sum = 0;
154     for(Cell cell : containerRef.get().cells) {
155       sum += cell.get();
156     }
157     return sum;
158   }
159 
160   @Override
161   public String toString() {
162     Cell[] cells = containerRef.get().cells;
163 
164     long min = Long.MAX_VALUE;
165     long max = Long.MIN_VALUE;
166     long sum = 0;
167 
168     for(Cell cell : cells) {
169       long value = cell.get();
170       sum += value;
171       if(min > value) { min = value; }
172       if(max < value) { max = value; }
173     }
174 
175     return new StringBuilder(100)
176     .append("[value=").append(sum)
177     .append(", cells=[length=").append(cells.length)
178     .append(", min=").append(min)
179     .append(", max=").append(max)
180     .append("]]").toString();
181   }
182 }