HBASE-8299 ExploringCompactionPolicy can get stuck in rare cases.
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1475966 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
eb192446a7
commit
b6aa74ef63
|
@ -26,10 +26,10 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.KeyValue.KVComparator;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
|
||||
import org.apache.hadoop.hbase.util.ReflectionUtils;
|
||||
import org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy;
|
||||
|
||||
/**
|
||||
* Default StoreEngine creates the default compactor, policy, and store file manager, or
|
||||
|
|
|
@ -280,6 +280,10 @@ public class HStore implements Store {
|
|||
public long getMemstoreFlushSize() {
|
||||
return this.region.memstoreFlushSize;
|
||||
}
|
||||
|
||||
public long getBlockingFileCount() {
|
||||
return blockingFileCount;
|
||||
}
|
||||
/* End implementation of StoreConfigInformation */
|
||||
|
||||
/**
|
||||
|
|
|
@ -40,4 +40,9 @@ public interface StoreConfigInformation {
|
|||
* Gets the cf-specific time-to-live for store files.
|
||||
*/
|
||||
public long getStoreFileTtl();
|
||||
|
||||
/**
|
||||
* The number of files required before flushes for this store will be blocked.
|
||||
*/
|
||||
public long getBlockingFileCount();
|
||||
}
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.apache.hadoop.hbase.regionserver.compactions;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
@ -29,80 +28,128 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
|
||||
/**
|
||||
* Class to pick which files if any to compact together.
|
||||
*
|
||||
* This class will search all possibilities for different and if it gets stuck it will choose
|
||||
* the smallest set of files to compact.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class ExploringCompactionPolicy extends RatioBasedCompactionPolicy {
|
||||
|
||||
public ExploringCompactionPolicy(Configuration conf,
|
||||
StoreConfigInformation storeConfigInfo) {
|
||||
/** Computed number of files that are needed to assume compactions are stuck. */
|
||||
private final long filesNeededToForce;
|
||||
|
||||
/**
|
||||
* Constructor for ExploringCompactionPolicy.
|
||||
* @param conf The configuration object
|
||||
* @param storeConfigInfo An object to provide info about the store.
|
||||
*/
|
||||
public ExploringCompactionPolicy(final Configuration conf,
|
||||
final StoreConfigInformation storeConfigInfo) {
|
||||
super(conf, storeConfigInfo);
|
||||
filesNeededToForce = storeConfigInfo.getBlockingFileCount();
|
||||
}
|
||||
|
||||
@Override
|
||||
ArrayList<StoreFile> applyCompactionPolicy(ArrayList<StoreFile> candidates,
|
||||
boolean mayUseOffPeak) throws IOException {
|
||||
final ArrayList<StoreFile> applyCompactionPolicy(final ArrayList<StoreFile> candidates,
|
||||
final boolean mayUseOffPeak) throws IOException {
|
||||
// Start off choosing nothing.
|
||||
List<StoreFile> bestSelection = new ArrayList<StoreFile>(0);
|
||||
List<StoreFile> smallest = new ArrayList<StoreFile>(0);
|
||||
long bestSize = 0;
|
||||
long smallestSize = Long.MAX_VALUE;
|
||||
|
||||
boolean mightBeStuck = candidates.size() >= filesNeededToForce;
|
||||
|
||||
// Consider every starting place.
|
||||
for (int start = 0; start < candidates.size(); start++) {
|
||||
// Consider every different sub list permutation in between start and end with min files.
|
||||
for(int currentEnd = start + comConf.getMinFilesToCompact() - 1;
|
||||
for (int currentEnd = start + comConf.getMinFilesToCompact() - 1;
|
||||
currentEnd < candidates.size(); currentEnd++) {
|
||||
List<StoreFile> potentialMatchFiles = candidates.subList(start, currentEnd+1);
|
||||
List<StoreFile> potentialMatchFiles = candidates.subList(start, currentEnd + 1);
|
||||
|
||||
// Sanity checks
|
||||
if (potentialMatchFiles.size() < comConf.getMinFilesToCompact()) continue;
|
||||
if (potentialMatchFiles.size() > comConf.getMaxFilesToCompact()) continue;
|
||||
if (!filesInRatio(potentialMatchFiles, mayUseOffPeak)) continue;
|
||||
if (potentialMatchFiles.size() < comConf.getMinFilesToCompact()) {
|
||||
continue;
|
||||
}
|
||||
if (potentialMatchFiles.size() > comConf.getMaxFilesToCompact()) {
|
||||
continue;
|
||||
}
|
||||
|
||||
// Compute the total size of files that will
|
||||
// have to be read if this set of files is compacted.
|
||||
long size = 0;
|
||||
long size = getTotalStoreSize(potentialMatchFiles);
|
||||
|
||||
for (StoreFile s:potentialMatchFiles) {
|
||||
size += s.getReader().length();
|
||||
// Store the smallest set of files. This stored set of files will be used
|
||||
// if it looks like the algorithm is stuck.
|
||||
if (size < smallestSize) {
|
||||
smallest = potentialMatchFiles;
|
||||
smallestSize = size;
|
||||
}
|
||||
|
||||
if (size >= comConf.getMinCompactSize()
|
||||
&& !filesInRatio(potentialMatchFiles, mayUseOffPeak)) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (size > comConf.getMaxCompactSize()) {
|
||||
continue;
|
||||
}
|
||||
|
||||
// Keep if this gets rid of more files. Or the same number of files for less io.
|
||||
if (potentialMatchFiles.size() > bestSelection.size() ||
|
||||
(potentialMatchFiles.size() == bestSelection.size() && size < bestSize)) {
|
||||
if (potentialMatchFiles.size() > bestSelection.size()
|
||||
|| (potentialMatchFiles.size() == bestSelection.size() && size < bestSize)) {
|
||||
bestSelection = potentialMatchFiles;
|
||||
bestSize = size;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (bestSelection.size() == 0 && mightBeStuck) {
|
||||
return new ArrayList<StoreFile>(smallest);
|
||||
}
|
||||
return new ArrayList<StoreFile>(bestSelection);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check that all files satisfy the r
|
||||
* @param files
|
||||
* @return
|
||||
* Find the total size of a list of store files.
|
||||
* @param potentialMatchFiles StoreFile list.
|
||||
* @return Sum of StoreFile.getReader().length();
|
||||
*/
|
||||
private boolean filesInRatio(List<StoreFile> files, boolean isOffPeak) {
|
||||
private long getTotalStoreSize(final List<StoreFile> potentialMatchFiles) {
|
||||
long size = 0;
|
||||
|
||||
for (StoreFile s:potentialMatchFiles) {
|
||||
size += s.getReader().length();
|
||||
}
|
||||
return size;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check that all files satisfy the constraint
|
||||
* FileSize(i) <= ( Sum(0,N,FileSize(_)) - FileSize(i) ) * Ratio.
|
||||
*
|
||||
* @param files List of store files to consider as a compaction candidate.
|
||||
* @param isOffPeak should the offPeak compaction ratio be used ?
|
||||
* @return a boolean if these files satisfy the ratio constraints.
|
||||
*/
|
||||
private boolean filesInRatio(final List<StoreFile> files, final boolean isOffPeak) {
|
||||
if (files.size() < 2) {
|
||||
return true;
|
||||
}
|
||||
double currentRatio = isOffPeak ?
|
||||
comConf.getCompactionRatioOffPeak() : comConf.getCompactionRatio();
|
||||
final double currentRatio =
|
||||
isOffPeak ? comConf.getCompactionRatioOffPeak() : comConf.getCompactionRatio();
|
||||
|
||||
long totalFileSize = 0;
|
||||
for (int i = 0; i < files.size(); i++) {
|
||||
totalFileSize += files.get(i).getReader().length();
|
||||
}
|
||||
for (int i = 0; i < files.size(); i++) {
|
||||
long singleFileSize = files.get(i).getReader().length();
|
||||
long sumAllOtherFilesize = totalFileSize - singleFileSize;
|
||||
long totalFileSize = getTotalStoreSize(files);
|
||||
|
||||
if (( singleFileSize > sumAllOtherFilesize * currentRatio)
|
||||
&& (sumAllOtherFilesize >= comConf.getMinCompactSize())){
|
||||
for (StoreFile file : files) {
|
||||
long singleFileSize = file.getReader().length();
|
||||
long sumAllOtherFileSizes = totalFileSize - singleFileSize;
|
||||
|
||||
if (singleFileSize > sumAllOtherFileSizes * currentRatio) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
return true;
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -114,6 +114,14 @@ public class RatioBasedCompactionPolicy extends CompactionPolicy {
|
|||
candidateSelection = checkMinFilesCriteria(candidateSelection);
|
||||
}
|
||||
candidateSelection = removeExcessFiles(candidateSelection, isUserCompaction, majorCompaction);
|
||||
|
||||
if (candidateSelection.size() == 0
|
||||
&& candidateFiles.size() >= storeConfigInfo.getBlockingFileCount()) {
|
||||
candidateSelection = new ArrayList<StoreFile>(candidateFiles);
|
||||
candidateSelection
|
||||
.subList(0, Math.max(0,candidateSelection.size() - comConf.getMinFilesToCompact()))
|
||||
.clear();
|
||||
}
|
||||
CompactionRequest result = new CompactionRequest(candidateSelection);
|
||||
result.setOffPeak(!candidateSelection.isEmpty() && !majorCompaction && mayUseOffPeak);
|
||||
return result;
|
||||
|
|
|
@ -62,7 +62,7 @@ public class TestDefaultCompactSelection extends TestCase {
|
|||
protected static final int maxFiles = 5;
|
||||
|
||||
protected static final long minSize = 10;
|
||||
protected static final long maxSize = 1000;
|
||||
protected static final long maxSize = 2100;
|
||||
|
||||
private HLog hlog;
|
||||
private HRegion region;
|
||||
|
@ -269,12 +269,8 @@ public class TestDefaultCompactSelection extends TestCase {
|
|||
// big size + threshold
|
||||
compactEquals(sfCreate(tooBig, tooBig, 700,700) /* empty */);
|
||||
// small files = don't care about ratio
|
||||
compactEquals(sfCreate(8,3,1), 8,3,1);
|
||||
/* TODO: add sorting + unit test back in when HBASE-2856 is fixed
|
||||
// sort first so you don't include huge file the tail end.
|
||||
// happens with HFileOutputFormat bulk migration
|
||||
compactEquals(sfCreate(100,50,23,12,12, 500), 23, 12, 12);
|
||||
*/
|
||||
compactEquals(sfCreate(7,1,1), 7,1,1);
|
||||
|
||||
// don't exceed max file compact threshold
|
||||
// note: file selection starts with largest to smallest.
|
||||
compactEquals(sfCreate(7, 6, 5, 4, 3, 2, 1), 5, 4, 3, 2, 1);
|
||||
|
@ -285,6 +281,15 @@ public class TestDefaultCompactSelection extends TestCase {
|
|||
|
||||
compactEquals(sfCreate(251, 253, 251, maxSize -1), 251, 253, 251);
|
||||
|
||||
compactEquals(sfCreate(maxSize -1,maxSize -1,maxSize -1) /* empty */);
|
||||
|
||||
// Always try and compact something to get below blocking storefile count
|
||||
this.conf.setLong("hbase.hstore.compaction.min.size", 1);
|
||||
store.storeEngine.getCompactionPolicy().setConf(conf);
|
||||
compactEquals(sfCreate(512,256,128,64,32,16,8,4,2,1), 4,2,1);
|
||||
this.conf.setLong("hbase.hstore.compaction.min.size", minSize);
|
||||
store.storeEngine.getCompactionPolicy().setConf(conf);
|
||||
|
||||
/* MAJOR COMPACTION */
|
||||
// if a major compaction has been forced, then compact everything
|
||||
compactEquals(sfCreate(50,25,12,12), true, 50, 25, 12, 12);
|
||||
|
|
|
@ -0,0 +1,65 @@
|
|||
/**
|
||||
* 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 java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
|
||||
/**
|
||||
* Class to generate several lists of StoreFiles that are all the same size.
|
||||
*/
|
||||
class ConstantSizeFileListGenerator extends StoreFileListGenerator {
|
||||
|
||||
/** How many mb's mock storefiles should be. */
|
||||
private static final int FILESIZE = 5;
|
||||
|
||||
ConstantSizeFileListGenerator() {
|
||||
super(ConstantSizeFileListGenerator.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Iterator<List<StoreFile>> iterator() {
|
||||
return new Iterator<List<StoreFile>>() {
|
||||
private int count = 0;
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return count < MAX_FILE_GEN_ITERS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<StoreFile> next() {
|
||||
count += 1;
|
||||
ArrayList<StoreFile> files = new ArrayList<StoreFile>(NUM_FILES_GEN);
|
||||
for (int i = 0; i < NUM_FILES_GEN; i++) {
|
||||
files.add(createMockStoreFile(FILESIZE));
|
||||
}
|
||||
return files;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -0,0 +1,54 @@
|
|||
/**
|
||||
* 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 java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
|
||||
/**
|
||||
* Test Policy to compact everything every time.
|
||||
*/
|
||||
public class EverythingPolicy extends RatioBasedCompactionPolicy {
|
||||
/**
|
||||
* Constructor.
|
||||
*
|
||||
* @param conf The Conf.
|
||||
* @param storeConfigInfo Info about the store.
|
||||
*/
|
||||
public EverythingPolicy(final Configuration conf,
|
||||
final StoreConfigInformation storeConfigInfo) {
|
||||
super(conf, storeConfigInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
final ArrayList<StoreFile> applyCompactionPolicy(final ArrayList<StoreFile> candidates,
|
||||
final boolean mayUseOffPeak) throws IOException {
|
||||
|
||||
if (candidates.size() < comConf.getMinFilesToCompact()) {
|
||||
return new ArrayList<StoreFile>(0);
|
||||
}
|
||||
|
||||
return new ArrayList<StoreFile>(candidates);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,76 @@
|
|||
/**
|
||||
* 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 java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
|
||||
/**
|
||||
* Class to create list of mock storefiles of specified length.
|
||||
* This is great for testing edge cases.
|
||||
*/
|
||||
class ExplicitFileListGenerator extends StoreFileListGenerator {
|
||||
/** The explicit files size lists to return. */
|
||||
private 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},
|
||||
{1, 1, 600, 1, 1, 1, 1},
|
||||
{1, 1, 600, 600, 600, 600, 600, 1, 1, 1, 1},
|
||||
{1, 1, 600, 600, 600, 1, 1, 1, 1},
|
||||
{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}
|
||||
};
|
||||
|
||||
ExplicitFileListGenerator() {
|
||||
super(ExplicitFileListGenerator.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Iterator<List<StoreFile>> iterator() {
|
||||
return new Iterator<List<StoreFile>>() {
|
||||
private int nextIndex = 0;
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return nextIndex < fileSizes.length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<StoreFile> next() {
|
||||
List<StoreFile> files = createStoreFileList(fileSizes[nextIndex]);
|
||||
nextIndex += 1;
|
||||
return files;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -0,0 +1,65 @@
|
|||
/**
|
||||
* 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 java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.commons.math.random.GaussianRandomGenerator;
|
||||
import org.apache.commons.math.random.MersenneTwister;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
|
||||
class GaussianFileListGenerator extends StoreFileListGenerator {
|
||||
|
||||
GaussianFileListGenerator() {
|
||||
super(GaussianFileListGenerator.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<StoreFile>> iterator() {
|
||||
return new Iterator<List<StoreFile>>() {
|
||||
private GaussianRandomGenerator gen =
|
||||
new GaussianRandomGenerator(new MersenneTwister(random.nextInt()));
|
||||
private int count = 0;
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return count < MAX_FILE_GEN_ITERS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<StoreFile> next() {
|
||||
count += 1;
|
||||
ArrayList<StoreFile> files = new ArrayList<StoreFile>(NUM_FILES_GEN);
|
||||
for (int i = 0; i < NUM_FILES_GEN; i++) {
|
||||
files.add(createMockStoreFile(
|
||||
(int) Math.ceil(Math.max(0, gen.nextNormalizedDouble() * 32 + 32)))
|
||||
);
|
||||
}
|
||||
|
||||
return files;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -0,0 +1,90 @@
|
|||
/**
|
||||
* 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 java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
import com.google.common.io.Files;
|
||||
import org.apache.commons.lang.RandomStringUtils;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
/**
|
||||
* Base class of objects that can create mock store files with a given size.
|
||||
*/
|
||||
class MockStoreFileGenerator {
|
||||
/** How many chars long the store file name will be. */
|
||||
private static final int FILENAME_LENGTH = 10;
|
||||
/** The random number generator. */
|
||||
protected Random random;
|
||||
|
||||
MockStoreFileGenerator(Class klass) {
|
||||
random = new Random(klass.getSimpleName().hashCode());
|
||||
}
|
||||
|
||||
protected List<StoreFile> createStoreFileList(final int[] fs) {
|
||||
List<StoreFile> storeFiles = new LinkedList<StoreFile>();
|
||||
for (int fileSize : fs) {
|
||||
storeFiles.add(createMockStoreFile(fileSize));
|
||||
}
|
||||
return storeFiles;
|
||||
}
|
||||
|
||||
protected StoreFile createMockStoreFile(final long size) {
|
||||
return createMockStoreFile(size * 1024 * 1024, -1L);
|
||||
}
|
||||
|
||||
protected StoreFile createMockStoreFileBytes(final long size) {
|
||||
return createMockStoreFile(size, -1L);
|
||||
}
|
||||
|
||||
protected StoreFile createMockStoreFile(final long sizeInBytes, final long seqId) {
|
||||
StoreFile mockSf = mock(StoreFile.class);
|
||||
StoreFile.Reader reader = mock(StoreFile.Reader.class);
|
||||
String stringPath = "/hbase/testTable/regionA/"
|
||||
+ RandomStringUtils.random(FILENAME_LENGTH, 0, 0, true, true, null, random);
|
||||
Path path = new Path(stringPath);
|
||||
|
||||
|
||||
when(reader.getSequenceID()).thenReturn(seqId);
|
||||
when(reader.getTotalUncompressedBytes()).thenReturn(sizeInBytes);
|
||||
when(reader.length()).thenReturn(sizeInBytes);
|
||||
|
||||
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", StringUtils.humanReadableInt(sizeInBytes))
|
||||
.add("seqId", seqId)
|
||||
.add("path", stringPath).toString();
|
||||
when(mockSf.toString()).thenReturn(toString);
|
||||
|
||||
return mockSf;
|
||||
}
|
||||
}
|
|
@ -18,14 +18,11 @@
|
|||
|
||||
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.MediumTests;
|
||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreConfigInformation;
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
|
@ -37,130 +34,135 @@ 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)
|
||||
@Category(MediumTests.class)
|
||||
@RunWith(Parameterized.class)
|
||||
public class PerfTestCompactionPolicies {
|
||||
public class PerfTestCompactionPolicies extends MockStoreFileGenerator {
|
||||
|
||||
static final Log LOG = LogFactory.getLog(PerfTestCompactionPolicies.class);
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(PerfTestCompactionPolicies.class);
|
||||
|
||||
private final RatioBasedCompactionPolicy cp;
|
||||
private final StoreFileListGenerator generator;
|
||||
private final HStore store;
|
||||
private Class<? extends StoreFileListGenerator> fileGenClass;
|
||||
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[][] {
|
||||
{RatioBasedCompactionPolicy.class, 3, 2, 1.2f},
|
||||
{ExploringCompactionPolicy.class, 3, 2, 1.2f},
|
||||
{RatioBasedCompactionPolicy.class, 4, 2, 1.2f},
|
||||
{ExploringCompactionPolicy.class, 4, 2, 1.2f},
|
||||
{RatioBasedCompactionPolicy.class, 5, 2, 1.2f},
|
||||
{ExploringCompactionPolicy.class, 5, 2, 1.2f},
|
||||
{RatioBasedCompactionPolicy.class, 4, 2, 1.3f},
|
||||
{ExploringCompactionPolicy.class, 4, 2, 1.3f},
|
||||
{RatioBasedCompactionPolicy.class, 4, 2, 1.4f},
|
||||
{ExploringCompactionPolicy.class, 4, 2, 1.4f},
|
||||
|
||||
});
|
||||
|
||||
|
||||
Class[] policyClasses = new Class[]{
|
||||
EverythingPolicy.class,
|
||||
RatioBasedCompactionPolicy.class,
|
||||
ExploringCompactionPolicy.class,
|
||||
};
|
||||
|
||||
Class[] fileListGenClasses = new Class[]{
|
||||
ExplicitFileListGenerator.class,
|
||||
ConstantSizeFileListGenerator.class,
|
||||
SemiConstantSizeFileListGenerator.class,
|
||||
GaussianFileListGenerator.class,
|
||||
SinusoidalFileListGenerator.class,
|
||||
SpikyFileListGenerator.class
|
||||
};
|
||||
|
||||
int[] maxFileValues = new int[] {10};
|
||||
int[] minFilesValues = new int[] {3};
|
||||
float[] ratioValues = new float[] {1.2f};
|
||||
|
||||
List<Object[]> params = new ArrayList<Object[]>(
|
||||
maxFileValues.length
|
||||
* minFilesValues.length
|
||||
* fileListGenClasses.length
|
||||
* policyClasses.length);
|
||||
|
||||
|
||||
for (Class policyClass : policyClasses) {
|
||||
for (Class genClass: fileListGenClasses) {
|
||||
for (int maxFile:maxFileValues) {
|
||||
for (int minFile:minFilesValues) {
|
||||
for (float ratio:ratioValues) {
|
||||
params.add(new Object[] {policyClass, genClass, maxFile, minFile, ratio});
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return params;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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.
|
||||
* Test the perf of a CompactionPolicy with settings.
|
||||
* @param cpClass The compaction policy to test
|
||||
* @param inMmax The maximum number of file to compact
|
||||
* @param inMin The min number of files to compact
|
||||
* @param inRatio The ratio that files must be under to be compacted.
|
||||
*/
|
||||
public PerfTestCompactionPolicies(Class<? extends CompactionPolicy> cpClass,
|
||||
int max, int min, float ratio) {
|
||||
this.max = max;
|
||||
this.min = min;
|
||||
this.ratio = ratio;
|
||||
public PerfTestCompactionPolicies(
|
||||
final Class<? extends CompactionPolicy> cpClass,
|
||||
final Class<? extends StoreFileListGenerator> fileGenClass,
|
||||
final int inMmax,
|
||||
final int inMin,
|
||||
final float inRatio) throws IllegalAccessException, InstantiationException {
|
||||
super(PerfTestCompactionPolicies.class);
|
||||
this.fileGenClass = fileGenClass;
|
||||
this.max = inMmax;
|
||||
this.min = inMin;
|
||||
this.ratio = inRatio;
|
||||
|
||||
//Hide lots of logging so the sysout is usable as a tab delimited file.
|
||||
// Hide lots of logging so the system out 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(RatioBasedCompactionPolicy.class).
|
||||
setLevel(org.apache.log4j.Level.ERROR);
|
||||
|
||||
org.apache.log4j.Logger.getLogger(cpClass).setLevel(org.apache.log4j.Level.ERROR);
|
||||
|
||||
|
||||
Configuration configuration = HBaseConfiguration.create();
|
||||
|
||||
//Make sure that this doesn't include every file.
|
||||
// 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);
|
||||
|
||||
HStore store = createMockStore();
|
||||
store = createMockStore();
|
||||
this.cp = ReflectionUtils.instantiateWithCustomCtor(cpClass.getName(),
|
||||
new Class[] { Configuration.class, StoreConfigInformation.class },
|
||||
new Object[] { configuration, store });
|
||||
new Class[] {Configuration.class, StoreConfigInformation.class },
|
||||
new Object[] {configuration, store });
|
||||
|
||||
//Used for making paths
|
||||
random = new Random(42);
|
||||
this.generator = fileGenClass.newInstance();
|
||||
// Used for making paths
|
||||
}
|
||||
|
||||
@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);
|
||||
public final void testSelection() throws Exception {
|
||||
long fileDiff = 0;
|
||||
for (List<StoreFile> storeFileList : generator) {
|
||||
List<StoreFile> currentFiles = new ArrayList<StoreFile>(18);
|
||||
for (StoreFile file : storeFileList) {
|
||||
currentFiles.add(file);
|
||||
currentFiles = runIteration(currentFiles);
|
||||
}
|
||||
fileDiff += (storeFileList.size() - currentFiles.size());
|
||||
}
|
||||
|
||||
//print out tab delimited so that it can be used in excel/gdocs.
|
||||
// print out tab delimited so that it can be used in excel/gdocs.
|
||||
System.out.println(
|
||||
cp.getClass().getSimpleName()
|
||||
cp.getClass().getSimpleName()
|
||||
+ "\t" + fileGenClass.getSimpleName()
|
||||
+ "\t" + max
|
||||
+ "\t" + min
|
||||
+ "\t" + ratio
|
||||
|
@ -175,7 +177,7 @@ public class PerfTestCompactionPolicies {
|
|||
List<StoreFile> storeFiles = new ArrayList<StoreFile>(startingStoreFiles);
|
||||
CompactionRequest req = cp.selectCompaction(
|
||||
storeFiles, new ArrayList<StoreFile>(), false, false, false);
|
||||
int newFileSize = 0;
|
||||
long newFileSize = 0;
|
||||
|
||||
Collection<StoreFile> filesToCompact = req.getFiles();
|
||||
|
||||
|
@ -188,55 +190,17 @@ public class PerfTestCompactionPolicies {
|
|||
newFileSize += storeFile.getReader().length();
|
||||
}
|
||||
|
||||
storeFiles.add(createMockStoreFile(newFileSize));
|
||||
storeFiles.add(createMockStoreFileBytes(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);
|
||||
when(s.getBlockingFileCount()).thenReturn(7L);
|
||||
return s;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,58 @@
|
|||
/**
|
||||
* 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 java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
|
||||
class SemiConstantSizeFileListGenerator extends StoreFileListGenerator {
|
||||
SemiConstantSizeFileListGenerator() {
|
||||
super(SemiConstantSizeFileListGenerator.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<StoreFile>> iterator() {
|
||||
return new Iterator<List<StoreFile>>() {
|
||||
private int count = 0;
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return count < MAX_FILE_GEN_ITERS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<StoreFile> next() {
|
||||
count += 1;
|
||||
ArrayList<StoreFile> files = new ArrayList<StoreFile>(NUM_FILES_GEN);
|
||||
for (int i = 0; i < NUM_FILES_GEN; i++) {
|
||||
files.add(createMockStoreFile(random.nextInt(5) + 30));
|
||||
}
|
||||
return files;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -0,0 +1,59 @@
|
|||
/**
|
||||
* 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 java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
|
||||
class SinusoidalFileListGenerator extends StoreFileListGenerator {
|
||||
SinusoidalFileListGenerator() {
|
||||
super(SinusoidalFileListGenerator.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<StoreFile>> iterator() {
|
||||
|
||||
|
||||
return new Iterator<List<StoreFile>>() {
|
||||
private int count = 0;
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return count < MAX_FILE_GEN_ITERS;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<StoreFile> next() {
|
||||
count += 1;
|
||||
ArrayList<StoreFile> files = new ArrayList<StoreFile>(NUM_FILES_GEN);
|
||||
for (int x = 0; x < NUM_FILES_GEN; x++) {
|
||||
int fileSize = (int) Math.abs(64 * Math.sin((Math.PI * x) / 50.0)) + 1;
|
||||
files.add(createMockStoreFile(fileSize));
|
||||
}
|
||||
return files;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -0,0 +1,62 @@
|
|||
/**
|
||||
* 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 java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
|
||||
class SpikyFileListGenerator extends StoreFileListGenerator {
|
||||
|
||||
SpikyFileListGenerator() {
|
||||
super(SpikyFileListGenerator.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<StoreFile>> iterator() {
|
||||
return new Iterator<List<StoreFile>>() {
|
||||
private int count = 0;
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return count < (MAX_FILE_GEN_ITERS);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<StoreFile> next() {
|
||||
count += 1;
|
||||
ArrayList<StoreFile> files = new ArrayList<StoreFile>(NUM_FILES_GEN);
|
||||
for (int x = 0; x < NUM_FILES_GEN; x++) {
|
||||
int fileSize = random.nextInt(5) + 1;
|
||||
if ( x % 10 == 0) {
|
||||
fileSize = random.nextInt(5) + 50;
|
||||
}
|
||||
files.add(createMockStoreFile(fileSize));
|
||||
}
|
||||
return files;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -0,0 +1,34 @@
|
|||
/**
|
||||
* 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 java.util.List;
|
||||
|
||||
import org.apache.hadoop.hbase.regionserver.StoreFile;
|
||||
|
||||
public abstract class StoreFileListGenerator
|
||||
extends MockStoreFileGenerator implements Iterable<List<StoreFile>> {
|
||||
|
||||
public static final int MAX_FILE_GEN_ITERS = 10;
|
||||
public static final int NUM_FILES_GEN = 1000;
|
||||
|
||||
StoreFileListGenerator(final Class klass) {
|
||||
super(klass);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue