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.io.hfile.bucket;
019
020import static org.junit.Assert.assertEquals;
021import static org.junit.Assert.assertFalse;
022import static org.junit.Assert.assertNull;
023import static org.junit.Assert.assertTrue;
024
025import java.io.FileNotFoundException;
026import java.io.IOException;
027import java.nio.ByteBuffer;
028import java.util.ArrayList;
029import java.util.Arrays;
030import java.util.List;
031import java.util.Map;
032import java.util.Random;
033import java.util.Set;
034import java.util.concurrent.locks.ReentrantReadWriteLock;
035import org.apache.hadoop.conf.Configuration;
036import org.apache.hadoop.fs.Path;
037import org.apache.hadoop.hbase.HBaseClassTestRule;
038import org.apache.hadoop.hbase.HBaseConfiguration;
039import org.apache.hadoop.hbase.HBaseTestingUtility;
040import org.apache.hadoop.hbase.HConstants;
041import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
042import org.apache.hadoop.hbase.io.hfile.BlockType;
043import org.apache.hadoop.hbase.io.hfile.CacheTestUtils;
044import org.apache.hadoop.hbase.io.hfile.CacheTestUtils.HFileBlockPair;
045import org.apache.hadoop.hbase.io.hfile.Cacheable;
046import org.apache.hadoop.hbase.io.hfile.HFileBlock;
047import org.apache.hadoop.hbase.io.hfile.HFileContext;
048import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
049import org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.BucketSizeInfo;
050import org.apache.hadoop.hbase.io.hfile.bucket.BucketAllocator.IndexStatistics;
051import org.apache.hadoop.hbase.io.hfile.bucket.BucketCache.RAMQueueEntry;
052import org.apache.hadoop.hbase.nio.ByteBuff;
053import org.apache.hadoop.hbase.testclassification.IOTests;
054import org.apache.hadoop.hbase.testclassification.MediumTests;
055import org.junit.After;
056import org.junit.Assert;
057import org.junit.Before;
058import org.junit.ClassRule;
059import org.junit.Test;
060import org.junit.experimental.categories.Category;
061import org.junit.runner.RunWith;
062import org.junit.runners.Parameterized;
063import org.mockito.Mockito;
064
065import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableMap;
066
067/**
068 * Basic test of BucketCache.Puts and gets.
069 * <p>
070 * Tests will ensure that blocks' data correctness under several threads concurrency
071 */
072@RunWith(Parameterized.class)
073@Category({ IOTests.class, MediumTests.class })
074public class TestBucketCache {
075
076  @ClassRule
077  public static final HBaseClassTestRule CLASS_RULE =
078      HBaseClassTestRule.forClass(TestBucketCache.class);
079
080  private static final Random RAND = new Random();
081
082  @Parameterized.Parameters(name = "{index}: blockSize={0}, bucketSizes={1}")
083  public static Iterable<Object[]> data() {
084    return Arrays.asList(new Object[][] {
085        { 8192, null }, // TODO: why is 8k the default blocksize for these tests?
086        {
087            16 * 1024,
088            new int[] { 2 * 1024 + 1024, 4 * 1024 + 1024, 8 * 1024 + 1024, 16 * 1024 + 1024,
089                28 * 1024 + 1024, 32 * 1024 + 1024, 64 * 1024 + 1024, 96 * 1024 + 1024,
090                128 * 1024 + 1024 } } });
091  }
092
093  @Parameterized.Parameter(0)
094  public int constructedBlockSize;
095
096  @Parameterized.Parameter(1)
097  public int[] constructedBlockSizes;
098
099  BucketCache cache;
100  final int CACHE_SIZE = 1000000;
101  final int NUM_BLOCKS = 100;
102  final int BLOCK_SIZE = CACHE_SIZE / NUM_BLOCKS;
103  final int NUM_THREADS = 100;
104  final int NUM_QUERIES = 10000;
105
106  final long capacitySize = 32 * 1024 * 1024;
107  final int writeThreads = BucketCache.DEFAULT_WRITER_THREADS;
108  final int writerQLen = BucketCache.DEFAULT_WRITER_QUEUE_ITEMS;
109  String ioEngineName = "offheap";
110  String persistencePath = null;
111
112  private static class MockedBucketCache extends BucketCache {
113
114    public MockedBucketCache(String ioEngineName, long capacity, int blockSize, int[] bucketSizes,
115        int writerThreads, int writerQLen, String persistencePath) throws FileNotFoundException,
116        IOException {
117      super(ioEngineName, capacity, blockSize, bucketSizes, writerThreads, writerQLen,
118          persistencePath);
119      super.wait_when_cache = true;
120    }
121
122    @Override
123    public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf, boolean inMemory) {
124      super.cacheBlock(cacheKey, buf, inMemory);
125    }
126
127    @Override
128    public void cacheBlock(BlockCacheKey cacheKey, Cacheable buf) {
129      super.cacheBlock(cacheKey, buf);
130    }
131  }
132
133  @Before
134  public void setup() throws FileNotFoundException, IOException {
135    cache =
136        new MockedBucketCache(ioEngineName, capacitySize, constructedBlockSize,
137            constructedBlockSizes, writeThreads, writerQLen, persistencePath);
138  }
139
140  @After
141  public void tearDown() {
142    cache.shutdown();
143  }
144
145  /**
146   * Return a random element from {@code a}.
147   */
148  private static <T> T randFrom(List<T> a) {
149    return a.get(RAND.nextInt(a.size()));
150  }
151
152  @Test
153  public void testBucketAllocator() throws BucketAllocatorException {
154    BucketAllocator mAllocator = cache.getAllocator();
155    /*
156     * Test the allocator first
157     */
158    final List<Integer> BLOCKSIZES = Arrays.asList(4 * 1024, 8 * 1024, 64 * 1024, 96 * 1024);
159
160    boolean full = false;
161    ArrayList<Long> allocations = new ArrayList<>();
162    // Fill the allocated extents by choosing a random blocksize. Continues selecting blocks until
163    // the cache is completely filled.
164    List<Integer> tmp = new ArrayList<>(BLOCKSIZES);
165    while (!full) {
166      Integer blockSize = null;
167      try {
168        blockSize = randFrom(tmp);
169        allocations.add(mAllocator.allocateBlock(blockSize));
170      } catch (CacheFullException cfe) {
171        tmp.remove(blockSize);
172        if (tmp.isEmpty()) full = true;
173      }
174    }
175
176    for (Integer blockSize : BLOCKSIZES) {
177      BucketSizeInfo bucketSizeInfo = mAllocator.roundUpToBucketSizeInfo(blockSize);
178      IndexStatistics indexStatistics = bucketSizeInfo.statistics();
179      assertEquals("unexpected freeCount for " + bucketSizeInfo, 0, indexStatistics.freeCount());
180    }
181
182    for (long offset : allocations) {
183      assertEquals(mAllocator.sizeOfAllocation(offset), mAllocator.freeBlock(offset));
184    }
185    assertEquals(0, mAllocator.getUsedSize());
186  }
187
188  @Test
189  public void testCacheSimple() throws Exception {
190    CacheTestUtils.testCacheSimple(cache, BLOCK_SIZE, NUM_QUERIES);
191  }
192
193  @Test
194  public void testCacheMultiThreadedSingleKey() throws Exception {
195    CacheTestUtils.hammerSingleKey(cache, BLOCK_SIZE, 2 * NUM_THREADS, 2 * NUM_QUERIES);
196  }
197
198  @Test
199  public void testHeapSizeChanges() throws Exception {
200    cache.stopWriterThreads();
201    CacheTestUtils.testHeapSizeChanges(cache, BLOCK_SIZE);
202  }
203
204  private void waitUntilFlushedToBucket(BucketCache cache, BlockCacheKey cacheKey)
205      throws InterruptedException {
206    while (!cache.backingMap.containsKey(cacheKey) || cache.ramCache.containsKey(cacheKey)) {
207      Thread.sleep(100);
208    }
209  }
210
211  // BucketCache.cacheBlock is async, it first adds block to ramCache and writeQueue, then writer
212  // threads will flush it to the bucket and put reference entry in backingMap.
213  private void cacheAndWaitUntilFlushedToBucket(BucketCache cache, BlockCacheKey cacheKey,
214      Cacheable block) throws InterruptedException {
215    cache.cacheBlock(cacheKey, block);
216    waitUntilFlushedToBucket(cache, cacheKey);
217  }
218
219  @Test
220  public void testMemoryLeak() throws Exception {
221    final BlockCacheKey cacheKey = new BlockCacheKey("dummy", 1L);
222    cacheAndWaitUntilFlushedToBucket(cache, cacheKey, new CacheTestUtils.ByteArrayCacheable(
223        new byte[10]));
224    long lockId = cache.backingMap.get(cacheKey).offset();
225    ReentrantReadWriteLock lock = cache.offsetLock.getLock(lockId);
226    lock.writeLock().lock();
227    Thread evictThread = new Thread("evict-block") {
228
229      @Override
230      public void run() {
231        cache.evictBlock(cacheKey);
232      }
233
234    };
235    evictThread.start();
236    cache.offsetLock.waitForWaiters(lockId, 1);
237    cache.blockEvicted(cacheKey, cache.backingMap.remove(cacheKey), true);
238    cacheAndWaitUntilFlushedToBucket(cache, cacheKey, new CacheTestUtils.ByteArrayCacheable(
239        new byte[10]));
240    lock.writeLock().unlock();
241    evictThread.join();
242    assertEquals(1L, cache.getBlockCount());
243    assertTrue(cache.getCurrentSize() > 0L);
244    assertTrue("We should have a block!", cache.iterator().hasNext());
245  }
246
247  @Test
248  public void testRetrieveFromFile() throws Exception {
249    HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
250    Path testDir = TEST_UTIL.getDataTestDir();
251    TEST_UTIL.getTestFileSystem().mkdirs(testDir);
252
253    BucketCache bucketCache = new BucketCache("file:" + testDir + "/bucket.cache", capacitySize,
254        constructedBlockSize, constructedBlockSizes, writeThreads, writerQLen, testDir
255            + "/bucket.persistence");
256    long usedSize = bucketCache.getAllocator().getUsedSize();
257    assertTrue(usedSize == 0);
258
259    HFileBlockPair[] blocks = CacheTestUtils.generateHFileBlocks(constructedBlockSize, 1);
260    // Add blocks
261    for (HFileBlockPair block : blocks) {
262      bucketCache.cacheBlock(block.getBlockName(), block.getBlock());
263    }
264    for (HFileBlockPair block : blocks) {
265      cacheAndWaitUntilFlushedToBucket(bucketCache, block.getBlockName(), block.getBlock());
266    }
267    usedSize = bucketCache.getAllocator().getUsedSize();
268    assertTrue(usedSize != 0);
269    // persist cache to file
270    bucketCache.shutdown();
271
272    // restore cache from file
273    bucketCache = new BucketCache("file:" + testDir + "/bucket.cache", capacitySize,
274        constructedBlockSize, constructedBlockSizes, writeThreads, writerQLen, testDir
275            + "/bucket.persistence");
276    assertEquals(usedSize, bucketCache.getAllocator().getUsedSize());
277    // persist cache to file
278    bucketCache.shutdown();
279
280    // reconfig buckets sizes, the biggest bucket is small than constructedBlockSize (8k or 16k)
281    // so it can't restore cache from file
282    int[] smallBucketSizes = new int[] { 2 * 1024 + 1024, 4 * 1024 + 1024 };
283    bucketCache = new BucketCache("file:" + testDir + "/bucket.cache", capacitySize,
284        constructedBlockSize, smallBucketSizes, writeThreads,
285        writerQLen, testDir + "/bucket.persistence");
286    assertEquals(0, bucketCache.getAllocator().getUsedSize());
287    assertEquals(0, bucketCache.backingMap.size());
288
289    TEST_UTIL.cleanupTestDir();
290  }
291
292  @Test
293  public void testBucketAllocatorLargeBuckets() throws BucketAllocatorException {
294    long availableSpace = 20 * 1024L * 1024 * 1024;
295    int[] bucketSizes = new int[]{1024, 1024 * 1024, 1024 * 1024 * 1024};
296    BucketAllocator allocator = new BucketAllocator(availableSpace, bucketSizes);
297    assertTrue(allocator.getBuckets().length > 0);
298  }
299
300  @Test
301  public void testGetPartitionSize() throws IOException {
302    //Test default values
303    validateGetPartitionSize(cache, BucketCache.DEFAULT_SINGLE_FACTOR, BucketCache.DEFAULT_MIN_FACTOR);
304
305    Configuration conf = HBaseConfiguration.create();
306    conf.setFloat(BucketCache.MIN_FACTOR_CONFIG_NAME, 0.5f);
307    conf.setFloat(BucketCache.SINGLE_FACTOR_CONFIG_NAME, 0.1f);
308    conf.setFloat(BucketCache.MULTI_FACTOR_CONFIG_NAME, 0.7f);
309    conf.setFloat(BucketCache.MEMORY_FACTOR_CONFIG_NAME, 0.2f);
310
311    BucketCache cache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
312        constructedBlockSizes, writeThreads, writerQLen, persistencePath, 100, conf);
313
314    validateGetPartitionSize(cache, 0.1f, 0.5f);
315    validateGetPartitionSize(cache, 0.7f, 0.5f);
316    validateGetPartitionSize(cache, 0.2f, 0.5f);
317  }
318
319  @Test
320  public void testValidBucketCacheConfigs() throws IOException {
321    Configuration conf = HBaseConfiguration.create();
322    conf.setFloat(BucketCache.ACCEPT_FACTOR_CONFIG_NAME, 0.9f);
323    conf.setFloat(BucketCache.MIN_FACTOR_CONFIG_NAME, 0.5f);
324    conf.setFloat(BucketCache.EXTRA_FREE_FACTOR_CONFIG_NAME, 0.5f);
325    conf.setFloat(BucketCache.SINGLE_FACTOR_CONFIG_NAME, 0.1f);
326    conf.setFloat(BucketCache.MULTI_FACTOR_CONFIG_NAME, 0.7f);
327    conf.setFloat(BucketCache.MEMORY_FACTOR_CONFIG_NAME, 0.2f);
328
329    BucketCache cache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
330        constructedBlockSizes, writeThreads, writerQLen, persistencePath, 100, conf);
331
332    assertEquals(BucketCache.ACCEPT_FACTOR_CONFIG_NAME + " failed to propagate.", 0.9f,
333        cache.getAcceptableFactor(), 0);
334    assertEquals(BucketCache.MIN_FACTOR_CONFIG_NAME + " failed to propagate.", 0.5f,
335        cache.getMinFactor(), 0);
336    assertEquals(BucketCache.EXTRA_FREE_FACTOR_CONFIG_NAME + " failed to propagate.", 0.5f,
337        cache.getExtraFreeFactor(), 0);
338    assertEquals(BucketCache.SINGLE_FACTOR_CONFIG_NAME + " failed to propagate.", 0.1f,
339        cache.getSingleFactor(), 0);
340    assertEquals(BucketCache.MULTI_FACTOR_CONFIG_NAME + " failed to propagate.", 0.7f,
341        cache.getMultiFactor(), 0);
342    assertEquals(BucketCache.MEMORY_FACTOR_CONFIG_NAME + " failed to propagate.", 0.2f,
343        cache.getMemoryFactor(), 0);
344  }
345
346  @Test
347  public void testInvalidAcceptFactorConfig() throws IOException {
348    float[] configValues = {-1f, 0.2f, 0.86f, 1.05f};
349    boolean[] expectedOutcomes = {false, false, true, false};
350    Map<String, float[]> configMappings = ImmutableMap.of(BucketCache.ACCEPT_FACTOR_CONFIG_NAME, configValues);
351    Configuration conf = HBaseConfiguration.create();
352    checkConfigValues(conf, configMappings, expectedOutcomes);
353  }
354
355  @Test
356  public void testInvalidMinFactorConfig() throws IOException {
357    float[] configValues = {-1f, 0f, 0.96f, 1.05f};
358    //throws due to <0, in expected range, minFactor > acceptableFactor, > 1.0
359    boolean[] expectedOutcomes = {false, true, false, false};
360    Map<String, float[]> configMappings = ImmutableMap
361      .of(BucketCache.MIN_FACTOR_CONFIG_NAME, configValues);
362    Configuration conf = HBaseConfiguration.create();
363    checkConfigValues(conf, configMappings, expectedOutcomes);
364  }
365
366  @Test
367  public void testInvalidExtraFreeFactorConfig() throws IOException {
368    float[] configValues = {-1f, 0f, 0.2f, 1.05f};
369    //throws due to <0, in expected range, in expected range, config can be > 1.0
370    boolean[] expectedOutcomes = {false, true, true, true};
371    Map<String, float[]> configMappings = ImmutableMap.of(BucketCache.EXTRA_FREE_FACTOR_CONFIG_NAME, configValues);
372    Configuration conf = HBaseConfiguration.create();
373    checkConfigValues(conf, configMappings, expectedOutcomes);
374  }
375
376  @Test
377  public void testInvalidCacheSplitFactorConfig() throws IOException {
378    float[] singleFactorConfigValues = {0.2f, 0f, -0.2f, 1f};
379    float[] multiFactorConfigValues = {0.4f, 0f, 1f, .05f};
380    float[] memoryFactorConfigValues = {0.4f, 0f, 0.2f, .5f};
381    // All configs add up to 1.0 and are between 0 and 1.0, configs don't add to 1.0, configs can't be negative, configs don't add to 1.0
382    boolean[] expectedOutcomes = {true, false, false, false};
383    Map<String, float[]> configMappings = ImmutableMap.of(BucketCache.SINGLE_FACTOR_CONFIG_NAME,
384        singleFactorConfigValues, BucketCache.MULTI_FACTOR_CONFIG_NAME, multiFactorConfigValues,
385        BucketCache.MEMORY_FACTOR_CONFIG_NAME, memoryFactorConfigValues);
386    Configuration conf = HBaseConfiguration.create();
387    checkConfigValues(conf, configMappings, expectedOutcomes);
388  }
389
390  private void checkConfigValues(Configuration conf, Map<String, float[]> configMap, boolean[] expectSuccess) throws IOException {
391    Set<String> configNames = configMap.keySet();
392    for (int i = 0; i < expectSuccess.length; i++) {
393      try {
394        for (String configName : configNames) {
395          conf.setFloat(configName, configMap.get(configName)[i]);
396        }
397        BucketCache cache = new BucketCache(ioEngineName, capacitySize, constructedBlockSize,
398            constructedBlockSizes, writeThreads, writerQLen, persistencePath, 100, conf);
399        assertTrue("Created BucketCache and expected it to succeed: " + expectSuccess[i] + ", but it actually was: " + !expectSuccess[i], expectSuccess[i]);
400      } catch (IllegalArgumentException e) {
401        assertFalse("Created BucketCache and expected it to succeed: " + expectSuccess[i] + ", but it actually was: " + !expectSuccess[i], expectSuccess[i]);
402      }
403    }
404  }
405
406  private void validateGetPartitionSize(BucketCache bucketCache, float partitionFactor, float minFactor) {
407    long expectedOutput = (long) Math.floor(bucketCache.getAllocator().getTotalSize() * partitionFactor * minFactor);
408    assertEquals(expectedOutput, bucketCache.getPartitionSize(partitionFactor));
409  }
410
411  @Test
412  public void testOffsetProducesPositiveOutput() {
413    //This number is picked because it produces negative output if the values isn't ensured to be positive.
414    //See HBASE-18757 for more information.
415    long testValue = 549888460800L;
416    BucketCache.BucketEntry bucketEntry = new BucketCache.BucketEntry(testValue, 10, 10L, true);
417    assertEquals(testValue, bucketEntry.offset());
418  }
419
420  @Test
421  public void testCacheBlockNextBlockMetadataMissing() throws Exception {
422    int size = 100;
423    int length = HConstants.HFILEBLOCK_HEADER_SIZE + size;
424    byte[] byteArr = new byte[length];
425    ByteBuffer buf = ByteBuffer.wrap(byteArr, 0, size);
426    HFileContext meta = new HFileContextBuilder().build();
427    HFileBlock blockWithNextBlockMetadata = new HFileBlock(BlockType.DATA, size, size, -1, buf,
428        HFileBlock.FILL_HEADER, -1, 52, -1, meta);
429    HFileBlock blockWithoutNextBlockMetadata = new HFileBlock(BlockType.DATA, size, size, -1, buf,
430        HFileBlock.FILL_HEADER, -1, -1, -1, meta);
431
432    BlockCacheKey key = new BlockCacheKey("key1", 0);
433    ByteBuffer actualBuffer = ByteBuffer.allocate(length);
434    ByteBuffer block1Buffer = ByteBuffer.allocate(length);
435    ByteBuffer block2Buffer = ByteBuffer.allocate(length);
436    blockWithNextBlockMetadata.serialize(block1Buffer, true);
437    blockWithoutNextBlockMetadata.serialize(block2Buffer, true);
438
439    // Add blockWithNextBlockMetadata, expect blockWithNextBlockMetadata back.
440    CacheTestUtils.getBlockAndAssertEquals(cache, key, blockWithNextBlockMetadata, actualBuffer,
441      block1Buffer);
442
443    waitUntilFlushedToBucket(cache, key);
444
445    // Add blockWithoutNextBlockMetada, expect blockWithNextBlockMetadata back.
446    CacheTestUtils.getBlockAndAssertEquals(cache, key, blockWithoutNextBlockMetadata, actualBuffer,
447      block1Buffer);
448
449    // Clear and add blockWithoutNextBlockMetadata
450    cache.evictBlock(key);
451    assertNull(cache.getBlock(key, false, false, false));
452    CacheTestUtils.getBlockAndAssertEquals(cache, key, blockWithoutNextBlockMetadata, actualBuffer,
453      block2Buffer);
454
455    waitUntilFlushedToBucket(cache, key);
456
457    // Add blockWithNextBlockMetadata, expect blockWithNextBlockMetadata to replace.
458    CacheTestUtils.getBlockAndAssertEquals(cache, key, blockWithNextBlockMetadata, actualBuffer,
459      block1Buffer);
460  }
461
462  @Test
463  public void testFreeBlockWhenIOEngineWriteFailure() throws IOException {
464    // initialize an block.
465    int size = 100, offset = 20;
466    int length = HConstants.HFILEBLOCK_HEADER_SIZE + size;
467    ByteBuffer buf = ByteBuffer.allocate(length);
468    HFileContext meta = new HFileContextBuilder().build();
469    HFileBlock block = new HFileBlock(BlockType.DATA, size, size, -1, buf, HFileBlock.FILL_HEADER,
470        offset, 52, -1, meta);
471
472    // initialize an mocked ioengine.
473    IOEngine ioEngine = Mockito.mock(IOEngine.class);
474    Mockito.when(ioEngine.usesSharedMemory()).thenReturn(false);
475    // Mockito.doNothing().when(ioEngine).write(Mockito.any(ByteBuffer.class), Mockito.anyLong());
476    Mockito.doThrow(RuntimeException.class).when(ioEngine).write(Mockito.any(ByteBuffer.class),
477      Mockito.anyLong());
478    Mockito.doThrow(RuntimeException.class).when(ioEngine).write(Mockito.any(ByteBuff.class),
479      Mockito.anyLong());
480
481    // create an bucket allocator.
482    long availableSpace = 1024 * 1024 * 1024L;
483    BucketAllocator allocator = new BucketAllocator(availableSpace, null);
484
485    BlockCacheKey key = new BlockCacheKey("dummy", 1L);
486    RAMQueueEntry re = new RAMQueueEntry(key, block, 1, true);
487
488    Assert.assertEquals(0, allocator.getUsedSize());
489    try {
490      re.writeToCache(ioEngine, allocator, new UniqueIndexMap<>(), null);
491      Assert.fail();
492    } catch (Exception e) {
493    }
494    Assert.assertEquals(0, allocator.getUsedSize());
495  }
496}