001/**
002 * Copyright The Apache Software Foundation
003 *
004 * Licensed to the Apache Software Foundation (ASF) under one
005 * or more contributor license agreements.  See the NOTICE file
006 * distributed with this work for additional information
007 * regarding copyright ownership.  The ASF licenses this file
008 * to you under the Apache License, Version 2.0 (the
009 * "License"); you may not use this file except in compliance
010 * with the License.  You may obtain a copy of the License at
011 *
012 *     http://www.apache.org/licenses/LICENSE-2.0
013 *
014 * Unless required by applicable law or agreed to in writing, software
015 * distributed under the License is distributed on an "AS IS" BASIS,
016 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
017 * See the License for the specific language governing permissions and
018 * limitations under the License.
019 */
020
021package org.apache.hadoop.hbase.util;
022
023import java.lang.reflect.Field;
024import java.lang.reflect.Modifier;
025import java.util.concurrent.ConcurrentHashMap;
026import java.util.concurrent.ConcurrentSkipListMap;
027
028import org.apache.yetus.audience.InterfaceAudience;
029import org.slf4j.Logger;
030import org.slf4j.LoggerFactory;
031
032/**
033 * Class for determining the "size" of a class, an attempt to calculate the
034 * actual bytes that an object of this class will occupy in memory
035 *
036 * The core of this class is taken from the Derby project
037 */
038@InterfaceAudience.Private
039public class ClassSize {
040  private static final Logger LOG = LoggerFactory.getLogger(ClassSize.class);
041
042  /** Array overhead */
043  public static final int ARRAY;
044
045  /** Overhead for ArrayList(0) */
046  public static final int ARRAYLIST;
047
048  /** Overhead for LinkedList(0) */
049  public static final int LINKEDLIST;
050
051  /** Overhead for a single entry in LinkedList */
052  public static final int LINKEDLIST_ENTRY;
053
054  /** Overhead for ByteBuffer */
055  public static final int BYTE_BUFFER;
056
057  /** Overhead for an Integer */
058  public static final int INTEGER;
059
060  /** Overhead for entry in map */
061  public static final int MAP_ENTRY;
062
063  /** Object overhead is minimum 2 * reference size (8 bytes on 64-bit) */
064  public static final int OBJECT;
065
066  /** Reference size is 8 bytes on 64-bit, 4 bytes on 32-bit */
067  public static final int REFERENCE;
068
069  /** String overhead */
070  public static final int STRING;
071
072  /** Overhead for TreeMap */
073  public static final int TREEMAP;
074
075  /** Overhead for ConcurrentHashMap */
076  public static final int CONCURRENT_HASHMAP;
077
078  /** Overhead for ConcurrentHashMap.Entry */
079  public static final int CONCURRENT_HASHMAP_ENTRY;
080
081  /** Overhead for ConcurrentHashMap.Segment */
082  public static final int CONCURRENT_HASHMAP_SEGMENT;
083
084  /** Overhead for ConcurrentSkipListMap */
085  public static final int CONCURRENT_SKIPLISTMAP;
086
087  /** Overhead for ConcurrentSkipListMap Entry */
088  public static final int CONCURRENT_SKIPLISTMAP_ENTRY;
089
090  /** Overhead for CellFlatMap */
091  public static final int CELL_FLAT_MAP;
092
093  /** Overhead for CellChunkMap */
094  public static final int CELL_CHUNK_MAP;
095
096  /** Overhead for Cell Chunk Map Entry */
097  public static final int CELL_CHUNK_MAP_ENTRY;
098
099  /** Overhead for CellArrayMap */
100  public static final int CELL_ARRAY_MAP;
101
102  /** Overhead for Cell Array Entry */
103  public static final int CELL_ARRAY_MAP_ENTRY;
104
105  /** Overhead for ReentrantReadWriteLock */
106  public static final int REENTRANT_LOCK;
107
108  /** Overhead for AtomicLong */
109  public static final int ATOMIC_LONG;
110
111  /** Overhead for AtomicInteger */
112  public static final int ATOMIC_INTEGER;
113
114  /** Overhead for AtomicBoolean */
115  public static final int ATOMIC_BOOLEAN;
116
117  /** Overhead for AtomicReference */
118  public static final int ATOMIC_REFERENCE;
119
120  /** Overhead for CopyOnWriteArraySet */
121  public static final int COPYONWRITE_ARRAYSET;
122
123  /** Overhead for CopyOnWriteArrayList */
124  public static final int COPYONWRITE_ARRAYLIST;
125
126  /** Overhead for timerange */
127  public static final int TIMERANGE;
128
129  /** Overhead for SyncTimeRangeTracker */
130  public static final int SYNC_TIMERANGE_TRACKER;
131
132  /** Overhead for NonSyncTimeRangeTracker */
133  public static final int NON_SYNC_TIMERANGE_TRACKER;
134
135  /** Overhead for CellSkipListSet */
136  public static final int CELL_SET;
137
138  public static final int STORE_SERVICES;
139
140  /**
141   * MemoryLayout abstracts details about the JVM object layout. Default implementation is used in
142   * case Unsafe is not available.
143   */
144  private static class MemoryLayout {
145    int headerSize() {
146      return 2 * oopSize();
147    }
148
149    int arrayHeaderSize() {
150      return (int) align(3 * oopSize());
151    }
152
153    /**
154     * Return the size of an "ordinary object pointer". Either 4 or 8, depending on 32/64 bit,
155     * and CompressedOops
156     */
157    int oopSize() {
158      return is32BitJVM() ? 4 : 8;
159    }
160
161    /**
162     * Aligns a number to 8.
163     * @param num number to align to 8
164     * @return smallest number >= input that is a multiple of 8
165     */
166    public long align(long num) {
167      //The 7 comes from that the alignSize is 8 which is the number of bytes
168      //stored and sent together
169      return  ((num + 7) >> 3) << 3;
170    }
171
172    long sizeOfByteArray(int len) {
173      return align(ARRAY + len);
174    }
175  }
176
177  /**
178   * UnsafeLayout uses Unsafe to guesstimate the object-layout related parameters like object header
179   * sizes and oop sizes
180   * See HBASE-15950.
181   */
182  private static class UnsafeLayout extends MemoryLayout {
183    @SuppressWarnings("unused")
184    private static final class HeaderSize {
185      private byte a;
186    }
187
188    public UnsafeLayout() {
189    }
190
191    @Override
192    int headerSize() {
193      try {
194        return (int) UnsafeAccess.theUnsafe.objectFieldOffset(
195          HeaderSize.class.getDeclaredField("a"));
196      } catch (NoSuchFieldException | SecurityException e) {
197        LOG.error(e.toString(), e);
198      }
199      return super.headerSize();
200    }
201
202    @Override
203    int arrayHeaderSize() {
204      return UnsafeAccess.theUnsafe.arrayBaseOffset(byte[].class);
205    }
206
207    @Override
208    @SuppressWarnings("static-access")
209    int oopSize() {
210      // Unsafe.addressSize() returns 8, even with CompressedOops. This is how many bytes each
211      // element is allocated in an Object[].
212      return UnsafeAccess.theUnsafe.ARRAY_OBJECT_INDEX_SCALE;
213    }
214
215    @Override
216    @SuppressWarnings("static-access")
217    long sizeOfByteArray(int len) {
218      return align(ARRAY + len * UnsafeAccess.theUnsafe.ARRAY_BYTE_INDEX_SCALE);
219    }
220  }
221
222  private static MemoryLayout getMemoryLayout() {
223    // Have a safeguard in case Unsafe estimate is wrong. This is static context, there is
224    // no configuration, so we look at System property.
225    String enabled = System.getProperty("hbase.memorylayout.use.unsafe");
226    if (UnsafeAvailChecker.isAvailable() && (enabled == null || Boolean.parseBoolean(enabled))) {
227      LOG.debug("Using Unsafe to estimate memory layout");
228      return new UnsafeLayout();
229    }
230    LOG.debug("Not using Unsafe to estimate memory layout");
231    return new MemoryLayout();
232  }
233
234  private static final MemoryLayout memoryLayout = getMemoryLayout();
235  private static final boolean USE_UNSAFE_LAYOUT = (memoryLayout instanceof UnsafeLayout);
236
237  public static boolean useUnsafeLayout() {
238    return USE_UNSAFE_LAYOUT;
239  }
240
241  /**
242   * Method for reading the arc settings and setting overheads according
243   * to 32-bit or 64-bit architecture.
244   */
245  static {
246    REFERENCE = memoryLayout.oopSize();
247
248    OBJECT = memoryLayout.headerSize();
249
250    ARRAY = memoryLayout.arrayHeaderSize();
251
252    ARRAYLIST = align(OBJECT + REFERENCE + (2 * Bytes.SIZEOF_INT)) + align(ARRAY);
253
254    LINKEDLIST = align(OBJECT + (2 * Bytes.SIZEOF_INT) + (2 * REFERENCE));
255
256    LINKEDLIST_ENTRY = align(OBJECT + (2 * REFERENCE));
257
258    //noinspection PointlessArithmeticExpression
259    BYTE_BUFFER = align(OBJECT + REFERENCE +
260        (5 * Bytes.SIZEOF_INT) +
261        (3 * Bytes.SIZEOF_BOOLEAN) + Bytes.SIZEOF_LONG) + align(ARRAY);
262
263    INTEGER = align(OBJECT + Bytes.SIZEOF_INT);
264
265    MAP_ENTRY = align(OBJECT + 5 * REFERENCE + Bytes.SIZEOF_BOOLEAN);
266
267    TREEMAP = align(OBJECT + (2 * Bytes.SIZEOF_INT) + 7 * REFERENCE);
268
269    // STRING is different size in jdk6 and jdk7. Just use what we estimate as size rather than
270    // have a conditional on whether jdk7.
271    STRING = (int) estimateBase(String.class, false);
272
273    // CONCURRENT_HASHMAP is different size in jdk6 and jdk7; it looks like its different between
274    // 23.6-b03 and 23.0-b21. Just use what we estimate as size rather than have a conditional on
275    // whether jdk7.
276    CONCURRENT_HASHMAP = (int) estimateBase(ConcurrentHashMap.class, false);
277
278    CONCURRENT_HASHMAP_ENTRY = align(REFERENCE + OBJECT + (3 * REFERENCE) +
279        (2 * Bytes.SIZEOF_INT));
280
281    CONCURRENT_HASHMAP_SEGMENT = align(REFERENCE + OBJECT +
282        (3 * Bytes.SIZEOF_INT) + Bytes.SIZEOF_FLOAT + ARRAY);
283
284    // The size changes from jdk7 to jdk8, estimate the size rather than use a conditional
285    CONCURRENT_SKIPLISTMAP = (int) estimateBase(ConcurrentSkipListMap.class, false);
286
287    // CellFlatMap object contains two integers, one boolean and one reference to object, so
288    // 2*INT + BOOLEAN + REFERENCE
289    CELL_FLAT_MAP = OBJECT + 2*Bytes.SIZEOF_INT + Bytes.SIZEOF_BOOLEAN + REFERENCE;
290
291    // CELL_ARRAY_MAP is the size of an instance of CellArrayMap class, which extends
292    // CellFlatMap class. CellArrayMap object containing a ref to an Array of Cells
293    CELL_ARRAY_MAP = align(CELL_FLAT_MAP + REFERENCE + ARRAY);
294
295    // CELL_CHUNK_MAP is the size of an instance of CellChunkMap class, which extends
296    // CellFlatMap class. CellChunkMap object containing a ref to an Array of Chunks
297    CELL_CHUNK_MAP = align(CELL_FLAT_MAP + REFERENCE + ARRAY);
298
299    CONCURRENT_SKIPLISTMAP_ENTRY = align(
300        align(OBJECT + (3 * REFERENCE)) + /* one node per entry */
301        align((OBJECT + (3 * REFERENCE))/2)); /* one index per two entries */
302
303    // REFERENCE in the CellArrayMap all the rest is counted in KeyValue.heapSize()
304    CELL_ARRAY_MAP_ENTRY = align(REFERENCE);
305
306    // The Cell Representation in the CellChunkMap, the Cell object size shouldn't be counted
307    // in KeyValue.heapSize()
308    // each cell-representation requires three integers for chunkID (reference to the ByteBuffer),
309    // offset and length, and one long for seqID
310    CELL_CHUNK_MAP_ENTRY = 3*Bytes.SIZEOF_INT + Bytes.SIZEOF_LONG;
311
312    REENTRANT_LOCK = align(OBJECT + (3 * REFERENCE));
313
314    ATOMIC_LONG = align(OBJECT + Bytes.SIZEOF_LONG);
315
316    ATOMIC_INTEGER = align(OBJECT + Bytes.SIZEOF_INT);
317
318    ATOMIC_BOOLEAN = align(OBJECT + Bytes.SIZEOF_BOOLEAN);
319
320    ATOMIC_REFERENCE = align(OBJECT + REFERENCE);
321
322    COPYONWRITE_ARRAYSET = align(OBJECT + REFERENCE);
323
324    COPYONWRITE_ARRAYLIST = align(OBJECT + (2 * REFERENCE) + ARRAY);
325
326    TIMERANGE = align(ClassSize.OBJECT + Bytes.SIZEOF_LONG * 2 + Bytes.SIZEOF_BOOLEAN);
327
328    SYNC_TIMERANGE_TRACKER = align(ClassSize.OBJECT + 2 * REFERENCE);
329
330    NON_SYNC_TIMERANGE_TRACKER = align(ClassSize.OBJECT + 2 * Bytes.SIZEOF_LONG);
331
332    CELL_SET = align(OBJECT + REFERENCE + Bytes.SIZEOF_INT);
333
334    STORE_SERVICES = align(OBJECT + REFERENCE + ATOMIC_LONG);
335  }
336
337  /**
338   * The estimate of the size of a class instance depends on whether the JVM
339   * uses 32 or 64 bit addresses, that is it depends on the size of an object
340   * reference. It is a linear function of the size of a reference, e.g.
341   * 24 + 5*r where r is the size of a reference (usually 4 or 8 bytes).
342   *
343   * This method returns the coefficients of the linear function, e.g. {24, 5}
344   * in the above example.
345   *
346   * @param cl A class whose instance size is to be estimated
347   * @param debug debug flag
348   * @return an array of 3 integers. The first integer is the size of the
349   * primitives, the second the number of arrays and the third the number of
350   * references.
351   */
352  @SuppressWarnings("unchecked")
353  private static int [] getSizeCoefficients(Class cl, boolean debug) {
354    int primitives = 0;
355    int arrays = 0;
356    int references = 0;
357    int index = 0;
358
359    for ( ; null != cl; cl = cl.getSuperclass()) {
360      Field[] field = cl.getDeclaredFields();
361      if (null != field) {
362        for (Field aField : field) {
363          if (Modifier.isStatic(aField.getModifiers())) continue;
364          Class fieldClass = aField.getType();
365          if (fieldClass.isArray()) {
366            arrays++;
367            references++;
368          } else if (!fieldClass.isPrimitive()) {
369            references++;
370          } else {// Is simple primitive
371            String name = fieldClass.getName();
372
373            if (name.equals("int") || name.equals("I"))
374              primitives += Bytes.SIZEOF_INT;
375            else if (name.equals("long") || name.equals("J"))
376              primitives += Bytes.SIZEOF_LONG;
377            else if (name.equals("boolean") || name.equals("Z"))
378              primitives += Bytes.SIZEOF_BOOLEAN;
379            else if (name.equals("short") || name.equals("S"))
380              primitives += Bytes.SIZEOF_SHORT;
381            else if (name.equals("byte") || name.equals("B"))
382              primitives += Bytes.SIZEOF_BYTE;
383            else if (name.equals("char") || name.equals("C"))
384              primitives += Bytes.SIZEOF_CHAR;
385            else if (name.equals("float") || name.equals("F"))
386              primitives += Bytes.SIZEOF_FLOAT;
387            else if (name.equals("double") || name.equals("D"))
388              primitives += Bytes.SIZEOF_DOUBLE;
389          }
390          if (debug) {
391            if (LOG.isDebugEnabled()) {
392              LOG.debug("" + index + " " + aField.getName() + " " + aField.getType());
393            }
394          }
395          index++;
396        }
397      }
398    }
399    return new int [] {primitives, arrays, references};
400  }
401
402  /**
403   * Estimate the static space taken up by a class instance given the
404   * coefficients returned by getSizeCoefficients.
405   *
406   * @param coeff the coefficients
407   *
408   * @param debug debug flag
409   * @return the size estimate, in bytes
410   */
411  private static long estimateBaseFromCoefficients(int [] coeff, boolean debug) {
412    long prealign_size = OBJECT + coeff[0] + coeff[2] * REFERENCE;
413
414    // Round up to a multiple of 8
415    long size = align(prealign_size) + align(coeff[1] * ARRAY);
416    if (debug) {
417      if (LOG.isDebugEnabled()) {
418        LOG.debug("Primitives=" + coeff[0] + ", arrays=" + coeff[1] +
419            ", references=" + coeff[2] + ", refSize " + REFERENCE +
420            ", size=" + size + ", prealign_size=" + prealign_size);
421      }
422    }
423    return size;
424  }
425
426  /**
427   * Estimate the static space taken up by the fields of a class. This includes
428   * the space taken up by by references (the pointer) but not by the referenced
429   * object. So the estimated size of an array field does not depend on the size
430   * of the array. Similarly the size of an object (reference) field does not
431   * depend on the object.
432   *
433   * @param cl class
434   * @param debug debug flag
435   * @return the size estimate in bytes.
436   */
437  @SuppressWarnings("unchecked")
438  public static long estimateBase(Class cl, boolean debug) {
439    return estimateBaseFromCoefficients( getSizeCoefficients(cl, debug), debug);
440  }
441
442  /**
443   * Aligns a number to 8.
444   * @param num number to align to 8
445   * @return smallest number &gt;= input that is a multiple of 8
446   */
447  public static int align(int num) {
448    return (int)(align((long)num));
449  }
450
451  /**
452   * Aligns a number to 8.
453   * @param num number to align to 8
454   * @return smallest number &gt;= input that is a multiple of 8
455   */
456  public static long align(long num) {
457    return memoryLayout.align(num);
458  }
459
460  /**
461   * Determines if we are running in a 32-bit JVM. Some unit tests need to
462   * know this too.
463   */
464  public static boolean is32BitJVM() {
465    final String model = System.getProperty("sun.arch.data.model");
466    return model != null && model.equals("32");
467  }
468
469  /**
470   * Calculate the memory consumption (in byte) of a byte array,
471   * including the array header and the whole backing byte array.
472   *
473   * If the whole byte array is occupied (not shared with other objects), please use this function.
474   * If not, please use {@link #sizeOfByteArray(int)} instead.
475   *
476   * @param b the byte array
477   * @return the memory consumption (in byte) of the whole byte array
478   */
479  public static long sizeOf(byte[] b) {
480    return memoryLayout.sizeOfByteArray(b.length);
481  }
482
483  /**
484   * Calculate the memory consumption (in byte) of a part of a byte array,
485   * including the array header and the part of the backing byte array.
486   *
487   * This function is used when the byte array backs multiple objects.
488   * For example, in {@link org.apache.hadoop.hbase.KeyValue},
489   * multiple KeyValue objects share a same backing byte array ({@link org.apache.hadoop.hbase.KeyValue#bytes}).
490   * Also see {@link org.apache.hadoop.hbase.KeyValue#heapSize()}.
491   *
492   * @param len the length (in byte) used partially in the backing byte array
493   * @return the memory consumption (in byte) of the part of the byte array
494   */
495  public static long sizeOfByteArray(int len) {
496    return memoryLayout.sizeOfByteArray(len);
497  }
498
499}
500