HBASE-7763 Compactions not sorting based on size anymore.
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1444977 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
27bc815ed5
commit
59a2adf714
|
@ -1793,18 +1793,19 @@ public class StoreFile {
|
||||||
/**
|
/**
|
||||||
* Useful comparators for comparing StoreFiles.
|
* Useful comparators for comparing StoreFiles.
|
||||||
*/
|
*/
|
||||||
abstract static class Comparators {
|
public abstract static class Comparators {
|
||||||
/**
|
/**
|
||||||
* Comparator that compares based on the Sequence Ids of the
|
* Comparator that compares based on the Sequence Ids of the
|
||||||
* the StoreFiles. Bulk loads that did not request a seq ID
|
* 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-
|
* 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,
|
* 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.
|
* If there are ties, the path name is used as a tie-breaker.
|
||||||
*/
|
*/
|
||||||
static final Comparator<StoreFile> SEQ_ID =
|
public static final Comparator<StoreFile> SEQ_ID =
|
||||||
Ordering.compound(ImmutableList.of(
|
Ordering.compound(ImmutableList.of(
|
||||||
Ordering.natural().onResultOf(new GetSeqId()),
|
Ordering.natural().onResultOf(new GetSeqId()),
|
||||||
|
Ordering.natural().onResultOf(new GetFileSize()).reverse(),
|
||||||
Ordering.natural().onResultOf(new GetBulkTime()),
|
Ordering.natural().onResultOf(new GetBulkTime()),
|
||||||
Ordering.natural().onResultOf(new GetPathName())
|
Ordering.natural().onResultOf(new GetPathName())
|
||||||
));
|
));
|
||||||
|
@ -1816,6 +1817,13 @@ public class StoreFile {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static class GetFileSize implements Function<StoreFile, Long> {
|
||||||
|
@Override
|
||||||
|
public Long apply(StoreFile sf) {
|
||||||
|
return sf.getReader().length();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private static class GetBulkTime implements Function<StoreFile, Long> {
|
private static class GetBulkTime implements Function<StoreFile, Long> {
|
||||||
@Override
|
@Override
|
||||||
public Long apply(StoreFile sf) {
|
public Long apply(StoreFile sf) {
|
||||||
|
@ -1830,19 +1838,5 @@ public class StoreFile {
|
||||||
return sf.getPath().getName();
|
return sf.getPath().getName();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* FILE_SIZE = descending sort StoreFiles (largest --> smallest in size)
|
|
||||||
*/
|
|
||||||
static final Comparator<StoreFile> FILE_SIZE = Ordering.natural().reverse()
|
|
||||||
.onResultOf(new Function<StoreFile, Long>() {
|
|
||||||
@Override
|
|
||||||
public Long apply(StoreFile sf) {
|
|
||||||
if (sf == null) {
|
|
||||||
throw new IllegalArgumentException("StorFile can not be null");
|
|
||||||
}
|
|
||||||
return sf.getReader().length();
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -589,13 +589,14 @@ public class TestStoreFile extends HBaseTestCase {
|
||||||
|
|
||||||
public void testSeqIdComparator() {
|
public void testSeqIdComparator() {
|
||||||
assertOrdering(StoreFile.Comparators.SEQ_ID,
|
assertOrdering(StoreFile.Comparators.SEQ_ID,
|
||||||
mockStoreFile(true, 1000, -1, "/foo/123"),
|
mockStoreFile(true, 100, 1000, -1, "/foo/123"),
|
||||||
mockStoreFile(true, 1000, -1, "/foo/126"),
|
mockStoreFile(true, 100, 1000, -1, "/foo/124"),
|
||||||
mockStoreFile(true, 2000, -1, "/foo/126"),
|
mockStoreFile(true, 99, 1000, -1, "/foo/126"),
|
||||||
mockStoreFile(false, -1, 1, "/foo/1"),
|
mockStoreFile(true, 98, 2000, -1, "/foo/126"),
|
||||||
mockStoreFile(false, -1, 3, "/foo/2"),
|
mockStoreFile(false, 3453, -1, 1, "/foo/1"),
|
||||||
mockStoreFile(false, -1, 5, "/foo/2"),
|
mockStoreFile(false, 2, -1, 3, "/foo/2"),
|
||||||
mockStoreFile(false, -1, 5, "/foo/3"));
|
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.
|
* Create a mock StoreFile with the given attributes.
|
||||||
*/
|
*/
|
||||||
private StoreFile mockStoreFile(boolean bulkLoad, long bulkTimestamp,
|
private StoreFile mockStoreFile(boolean bulkLoad,
|
||||||
long seqId, String path) {
|
long size,
|
||||||
|
long bulkTimestamp,
|
||||||
|
long seqId,
|
||||||
|
String path) {
|
||||||
StoreFile mock = Mockito.mock(StoreFile.class);
|
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(bulkLoad).when(mock).isBulkLoadResult();
|
||||||
Mockito.doReturn(bulkTimestamp).when(mock).getBulkLoadTimestamp();
|
Mockito.doReturn(bulkTimestamp).when(mock).getBulkLoadTimestamp();
|
||||||
Mockito.doReturn(seqId).when(mock).getMaxSequenceId();
|
Mockito.doReturn(seqId).when(mock).getMaxSequenceId();
|
||||||
|
|
|
@ -0,0 +1,239 @@
|
||||||
|
/**
|
||||||
|
* 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 com.google.common.base.Objects;
|
||||||
|
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 PerfTestCompactionPolicies {
|
||||||
|
|
||||||
|
static final Log LOG = LogFactory.getLog(PerfTestCompactionPolicies.class);
|
||||||
|
|
||||||
|
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<Object[]> data() {
|
||||||
|
return Arrays.asList(new Object[][] {
|
||||||
|
{new DefaultCompactionPolicy(), 3, 2, 1.2f},
|
||||||
|
{new DefaultCompactionPolicy(), 4, 2, 1.2f},
|
||||||
|
{new DefaultCompactionPolicy(), 5, 2, 1.2f},
|
||||||
|
{new DefaultCompactionPolicy(), 4, 2, 1.3f},
|
||||||
|
{new DefaultCompactionPolicy(), 4, 2, 1.4f},
|
||||||
|
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Test the perf of a CompactionPolicy with settings
|
||||||
|
* @param cp The compaction policy to test
|
||||||
|
* @param max The maximum number of file to compact
|
||||||
|
* @param min The min number of files to compact
|
||||||
|
* @param ratio The ratio that files must be under to be compacted.
|
||||||
|
*/
|
||||||
|
public PerfTestCompactionPolicies(CompactionPolicy cp, int max, int min, float ratio) {
|
||||||
|
this.max = max;
|
||||||
|
this.min = min;
|
||||||
|
this.ratio = ratio;
|
||||||
|
|
||||||
|
//Hide lots of logging so the sysout is usable as a tab delimited file.
|
||||||
|
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.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. To simulate bulk loading patterns.
|
||||||
|
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},
|
||||||
|
{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 15}
|
||||||
|
};
|
||||||
|
|
||||||
|
for (int[] fs : fileSizes) {
|
||||||
|
List<StoreFile> storeFiles = createStoreFileList(fs);
|
||||||
|
storeFiles = runIteration(storeFiles);
|
||||||
|
runIteration(storeFiles);
|
||||||
|
}
|
||||||
|
|
||||||
|
for (int i = 0; i < 100; i++) {
|
||||||
|
List<StoreFile> storeFiles = new LinkedList<StoreFile>();
|
||||||
|
|
||||||
|
//Add some files to start with so that things are more normal
|
||||||
|
storeFiles.add(createMockStoreFile(random.nextInt(1700) + 500));
|
||||||
|
storeFiles.add(createMockStoreFile(random.nextInt(700) + 400));
|
||||||
|
storeFiles.add(createMockStoreFile(random.nextInt(400) + 300));
|
||||||
|
storeFiles.add(createMockStoreFile(random.nextInt(400) + 200));
|
||||||
|
|
||||||
|
for (int x = 0; x < 50; x++) {
|
||||||
|
storeFiles.add(createMockStoreFile(random.nextInt(90) + 10));
|
||||||
|
storeFiles.add(createMockStoreFile(random.nextInt(90) + 10));
|
||||||
|
storeFiles.add(createMockStoreFile(random.nextInt(90) + 10));
|
||||||
|
storeFiles.add(createMockStoreFile(random.nextInt(90) + 10));
|
||||||
|
storeFiles.add(createMockStoreFile(random.nextInt(90) + 10));
|
||||||
|
storeFiles.add(createMockStoreFile(random.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(
|
||||||
|
cp.getClass().getSimpleName()
|
||||||
|
+ "\t" + max
|
||||||
|
+ "\t" + min
|
||||||
|
+ "\t" + ratio
|
||||||
|
+ "\t" + written
|
||||||
|
+ "\t" + fileDiff
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
private List<StoreFile> runIteration(List<StoreFile> startingStoreFiles) throws IOException {
|
||||||
|
|
||||||
|
List<StoreFile> storeFiles = new ArrayList<StoreFile>(startingStoreFiles);
|
||||||
|
CompactSelection sel = cp.selectCompaction(storeFiles, false, false);
|
||||||
|
int newFileSize = 0;
|
||||||
|
|
||||||
|
List<StoreFile> filesToCompact = sel.getFilesToCompact();
|
||||||
|
|
||||||
|
if (!filesToCompact.isEmpty()) {
|
||||||
|
|
||||||
|
storeFiles = new ArrayList<StoreFile>(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<StoreFile> createStoreFileList(int[] fs) {
|
||||||
|
List<StoreFile> storeFiles = new LinkedList<StoreFile>();
|
||||||
|
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.isReference()).thenReturn(false); // TODO come back to
|
||||||
|
// this when selection takes this into account
|
||||||
|
when(mockSf.getReader()).thenReturn(reader);
|
||||||
|
String toString = Objects.toStringHelper("MockStoreFile")
|
||||||
|
.add("isReference", false)
|
||||||
|
.add("fileSize", sizeMb)
|
||||||
|
.add("seqId", seqId)
|
||||||
|
.add("path", stringPath).toString();
|
||||||
|
when(mockSf.toString()).thenReturn(toString);
|
||||||
|
|
||||||
|
return mockSf;
|
||||||
|
}
|
||||||
|
|
||||||
|
private HStore createMockStore() {
|
||||||
|
HStore s = mock(HStore.class);
|
||||||
|
when(s.getStoreFileTtl()).thenReturn(Long.MAX_VALUE);
|
||||||
|
return s;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
Loading…
Reference in New Issue