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