From e5fa45fd02377c1a01316e703e39d98634340ba8 Mon Sep 17 00:00:00 2001 From: Elliott Clark Date: Tue, 5 Feb 2013 11:12:56 -0800 Subject: [PATCH] Compactions not sorting based on size anymore. --- .../hadoop/hbase/regionserver/StoreFile.java | 28 +-- .../hadoop/hbase/regionserver/TestStoreFile.java | 27 ++- .../compactions/PerfTestCompactionPolicy.java | 252 ++++++++++++++++++++ 3 files changed, 281 insertions(+), 26 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicy.java diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java index 3eb6d25..36e3bbc 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java @@ -1761,18 +1761,19 @@ public class StoreFile { /** * Useful comparators for comparing StoreFiles. */ - abstract static class Comparators { + public abstract static class Comparators { /** * Comparator that compares based on the Sequence Ids of the * the StoreFiles. Bulk loads that did not request a seq ID * are given a seq id of -1; thus, they are placed before all non- * bulk loads, and bulk loads with sequence Id. Among these files, - * the bulkLoadTime is used to determine the ordering. + * the size is used to determine the ordering, then bulkLoadTime. * If there are ties, the path name is used as a tie-breaker. */ - static final Comparator SEQ_ID = + public static final Comparator SEQ_ID = Ordering.compound(ImmutableList.of( Ordering.natural().onResultOf(new GetSeqId()), + Ordering.natural().onResultOf(new GetFileSize()).reverse(), Ordering.natural().onResultOf(new GetBulkTime()), Ordering.natural().onResultOf(new GetPathName()) )); @@ -1784,6 +1785,13 @@ public class StoreFile { } } + private static class GetFileSize implements Function { + @Override + public Long apply(StoreFile sf) { + return sf.getReader().length(); + } + } + private static class GetBulkTime implements Function { @Override public Long apply(StoreFile sf) { @@ -1798,19 +1806,5 @@ public class StoreFile { return sf.getPath().getName(); } } - - /** - * FILE_SIZE = descending sort StoreFiles (largest --> smallest in size) - */ - static final Comparator FILE_SIZE = Ordering.natural().reverse() - .onResultOf(new Function() { - @Override - public Long apply(StoreFile sf) { - if (sf == null) { - throw new IllegalArgumentException("StorFile can not be null"); - } - return sf.getReader().length(); - } - }); } } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java index 6d58ae7..6b3e2d5 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStoreFile.java @@ -589,13 +589,14 @@ public class TestStoreFile extends HBaseTestCase { public void testSeqIdComparator() { assertOrdering(StoreFile.Comparators.SEQ_ID, - mockStoreFile(true, 1000, -1, "/foo/123"), - mockStoreFile(true, 1000, -1, "/foo/126"), - mockStoreFile(true, 2000, -1, "/foo/126"), - mockStoreFile(false, -1, 1, "/foo/1"), - mockStoreFile(false, -1, 3, "/foo/2"), - mockStoreFile(false, -1, 5, "/foo/2"), - mockStoreFile(false, -1, 5, "/foo/3")); + mockStoreFile(true, 100, 1000, -1, "/foo/123"), + mockStoreFile(true, 100, 1000, -1, "/foo/124"), + mockStoreFile(true, 99, 1000, -1, "/foo/126"), + mockStoreFile(true, 98, 2000, -1, "/foo/126"), + mockStoreFile(false, 3453, -1, 1, "/foo/1"), + mockStoreFile(false, 2, -1, 3, "/foo/2"), + mockStoreFile(false, 1000, -1, 5, "/foo/2"), + mockStoreFile(false, 76, -1, 5, "/foo/3")); } /** @@ -614,9 +615,17 @@ public class TestStoreFile extends HBaseTestCase { /** * Create a mock StoreFile with the given attributes. */ - private StoreFile mockStoreFile(boolean bulkLoad, long bulkTimestamp, - long seqId, String path) { + private StoreFile mockStoreFile(boolean bulkLoad, + long size, + long bulkTimestamp, + long seqId, + String path) { StoreFile mock = Mockito.mock(StoreFile.class); + StoreFile.Reader reader = Mockito.mock(StoreFile.Reader.class); + + Mockito.doReturn(size).when(reader).length(); + + Mockito.doReturn(reader).when(mock).getReader(); Mockito.doReturn(bulkLoad).when(mock).isBulkLoadResult(); Mockito.doReturn(bulkTimestamp).when(mock).getBulkLoadTimestamp(); Mockito.doReturn(seqId).when(mock).getMaxSequenceId(); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicy.java hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicy.java new file mode 100644 index 0000000..207c205 --- /dev/null +++ hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/PerfTestCompactionPolicy.java @@ -0,0 +1,252 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.regionserver.compactions; + +import org.apache.commons.lang.RandomStringUtils; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.SmallTests; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.LinkedList; +import java.util.List; +import java.util.Random; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@Category(SmallTests.class) +@RunWith(Parameterized.class) +public class PerfTestCompactionPolicy { + + static final Log LOG = LogFactory.getLog(PerfTestCompactionPolicy.class); + + private final String name; + private final CompactionPolicy cp; + private final int max; + private final int min; + private final float ratio; + private long written = 0; + private long fileDiff = 0; + private Random random; + + @Parameterized.Parameters + public static Collection data() { + return Arrays.asList(new Object[][] { + {"Default", new DefaultCompactionPolicy(), 3, 2, 1.2f}, + {"Default", new DefaultCompactionPolicy(), 4, 2, 1.2f}, + {"Default", new DefaultCompactionPolicy(), 5, 2, 1.2f}, + {"Default", new DefaultCompactionPolicy(), 4, 2, 1.3f}, + {"Default", new DefaultCompactionPolicy(), 4, 2, 1.4f}, + + }); + } + + public PerfTestCompactionPolicy(String name, CompactionPolicy cp, int max, int min, float ratio) { + this.max = max; + this.min = min; + this.ratio = ratio; + + org.apache.log4j.Logger.getLogger(CompactionConfiguration.class). + setLevel(org.apache.log4j.Level.ERROR); + + org.apache.log4j.Logger.getLogger(cp.getClass()). + setLevel(org.apache.log4j.Level.ERROR); + + this.name = name; + this.cp = cp; + + + Configuration configuration = HBaseConfiguration.create(); + + //Make sure that this doesn't include every file. + configuration.setInt("hbase.hstore.compaction.max", max); + configuration.setInt("hbase.hstore.compaction.min", min); + configuration.setFloat("hbase.hstore.compaction.ratio", ratio); + + cp.store = createMockStore(); + + //Now set the conf. + cp.setConf(configuration); + + + //Used for making paths + random = new Random(42); + } + + @Test + public void testSelection() throws Exception { + //Some special cases. + int[][] fileSizes = new int[][] { + {1000,350,200,100,20,10,10}, + {1000,450,200,100,20,10,10}, + {1000,550,200,100,20,10,10}, + {1000,650,200,100,20,10,10}, + {1000,250,25,25,25,25,25,25}, + {25,25,25,25,25,25, 500}, + {1000,1000,1000,1000,900}, + {107, 50, 10, 10, 10, 10}, + {2000, 107, 50, 10, 10, 10, 10}, + {9,8,7,6,5,4,3,2,1}, + {11,18,9,8,7,6,5,4,3,2,1}, + {110,18,18,18,18,9,8,7,6,5,4,3,2,1} + }; + + for(int[] fs:fileSizes) { + List storeFiles = createStoreFileList(fs); + storeFiles = runIteration(storeFiles); + storeFiles = runIteration(storeFiles); + } + + // Bunch of random seed numbers. + // They just allow pretty easy number + // generation that is repeatable. + long[] seeds = new long[] { + -71810, 43328, 45852, -97821, 72864, + -27902, -9832, 92684, 4900, 25394, + -30447, 99580, -25731, -22047, 81157, + 66516, -99093, -48139, -82778, 60701, + 41455, 82852, -91176, -97101, 37952, + 61178, 19021, -64174, 10997, -66468, + -75501, -41283, -79849, -30542, 89635, + -90500, 32912, -11541, 44923, -71162, + -56932, -98994, -78222, 85530, -15455, + -79379, 42617, 19568, -72522, 57925, + 7131, -90911, 45214, 60855, 58307, + -60695, -12032, 12846, 42639, 84767, + -88972, -89194, -22965, -29059, -82397, + 72363, -53877, 8648, -59686, 94827, + -7513, -97734, 72303, -21442, -84681, + -34609, -8956, 3613, 69288, -16987, + 81842, -63248, 9332, -90234, 51453, + -3835, 36257, 92924, -34882, -94294, + 2611, -99898, 6331, 7079, 53965, + -87720, -16930, 95989, 2361, 83507, + -93274, 31224, 16772, -31809, 17517}; + + for (long seed:seeds) { + Random fsRandom = new Random(seed); + List storeFiles = new LinkedList(); + + //Add some files to start with so that things are more normal + storeFiles.add(createMockStoreFile(fsRandom.nextInt(1700) + 500)); + storeFiles.add(createMockStoreFile(fsRandom.nextInt(700) + 400)); + storeFiles.add(createMockStoreFile(fsRandom.nextInt(400) + 300)); + storeFiles.add(createMockStoreFile(fsRandom.nextInt(400) + 200)); + + for (int i =0; i< 50; i++) { + storeFiles.add(createMockStoreFile(fsRandom.nextInt(90) + 10)); + storeFiles.add(createMockStoreFile(fsRandom.nextInt(90) + 10)); + storeFiles.add(createMockStoreFile(fsRandom.nextInt(90) + 10)); + storeFiles.add(createMockStoreFile(fsRandom.nextInt(90) + 10)); + storeFiles.add(createMockStoreFile(fsRandom.nextInt(90) + 10)); + storeFiles.add(createMockStoreFile(fsRandom.nextInt(90) + 10)); + storeFiles = runIteration(storeFiles); + storeFiles = runIteration(storeFiles); + } + } + + //print out tab delimited so that it can be used in excel/gdocs. + System.out.println( + name + + "\t" + max + + "\t" + min + + "\t" + ratio + + "\t" + written + + "\t" + fileDiff + ); + } + + + private List runIteration(List startingStoreFiles) throws IOException { + + List storeFiles = new ArrayList(startingStoreFiles); + CompactSelection sel = cp.selectCompaction(new ArrayList(storeFiles), false, false); + int newFileSize = 0; + + List filesToCompact = sel.getFilesToCompact(); + + if (!filesToCompact.isEmpty()) { + + + storeFiles = new ArrayList(storeFiles); + storeFiles.removeAll(filesToCompact); + + for (StoreFile storeFile : filesToCompact) { + newFileSize += storeFile.getReader().length(); + } + storeFiles.add(createMockStoreFile(newFileSize)); + } + + written += newFileSize; + fileDiff += storeFiles.size() - startingStoreFiles.size(); + return storeFiles; + } + + private List createStoreFileList(int[] fs) { + List storeFiles = new LinkedList(); + for(int fileSize:fs) { + storeFiles.add(createMockStoreFile(fileSize)); + } + return storeFiles; + } + + private StoreFile createMockStoreFile(int sizeMb) { + return createMockStoreFile(sizeMb, -1l); + } + + + private StoreFile createMockStoreFile(int sizeMb, long seqId) { + StoreFile mockSf = mock(StoreFile.class); + StoreFile.Reader reader = mock(StoreFile.Reader.class); + String stringPath = "/hbase/"+ RandomStringUtils.random(10, 0, 0, true, true, null, random); + Path path = new Path(stringPath); + + when(reader.getSequenceID()).thenReturn(seqId); + when(reader.getTotalUncompressedBytes()).thenReturn(Long.valueOf(sizeMb)); + when(reader.length()).thenReturn(Long.valueOf(sizeMb)); + + when(mockSf.getPath()).thenReturn(path); + when(mockSf.excludeFromMinorCompaction()).thenReturn(false); + when(mockSf.getReader()).thenReturn(reader); + when(mockSf.toString()).thenReturn("MockStoreFile File Size: " + sizeMb + " seqId: "+seqId); + + return mockSf; + } + + private HStore createMockStore() { + HStore s = mock(HStore.class); + when(s.getStoreFileTtl()).thenReturn(Long.MAX_VALUE); + + return s; + } + +} -- 1.7.10.2 (Apple Git-33)