// only to be used by init(), to setup the very first memtable for the cfs
 public Memtable(AtomicReference<ReplayPosition> commitLogLowerBound, ColumnFamilyStore cfs) {
   this.cfs = cfs;
   this.commitLogLowerBound = commitLogLowerBound;
   this.allocator = MEMORY_POOL.newAllocator();
   this.initialComparator = cfs.metadata.comparator;
   this.cfs.scheduleFlush();
   this.columnsCollector = new ColumnsCollector(cfs.metadata.partitionColumns());
 }
 private static int estimateRowOverhead(final int count) {
   // calculate row overhead
   try (final OpOrder.Group group = new OpOrder().start()) {
     int rowOverhead;
     MemtableAllocator allocator = MEMORY_POOL.newAllocator();
     ConcurrentNavigableMap<PartitionPosition, Object> partitions = new ConcurrentSkipListMap<>();
     final Object val = new Object();
     for (int i = 0; i < count; i++)
       partitions.put(
           allocator.clone(
               new BufferDecoratedKey(new LongToken(i), ByteBufferUtil.EMPTY_BYTE_BUFFER), group),
           val);
     double avgSize = ObjectSizes.measureDeep(partitions) / (double) count;
     rowOverhead =
         (int) ((avgSize - Math.floor(avgSize)) < 0.05 ? Math.floor(avgSize) : Math.ceil(avgSize));
     rowOverhead -= ObjectSizes.measureDeep(new LongToken(0));
     rowOverhead += AtomicBTreePartition.EMPTY_SIZE;
     allocator.setDiscarding();
     allocator.setDiscarded();
     return rowOverhead;
   }
 }