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