1
0

Throttling to limit QPS from HbaseIndex

This commit is contained in:
Kaushik Devarajaiah
2018-07-19 13:46:33 -07:00
committed by vinoth chandar
parent 3746ace76a
commit e624480259
6 changed files with 436 additions and 50 deletions

View File

@@ -27,14 +27,18 @@ import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieCompactionConfig;
import com.uber.hoodie.config.HoodieHBaseIndexConfig;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieStorageConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.hbase.HBaseIndex;
import com.uber.hoodie.index.hbase.HBaseIndex.HbasePutBatchSizeCalculator;
import com.uber.hoodie.table.HoodieTable;
import java.io.File;
import java.util.Arrays;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@@ -51,6 +55,7 @@ import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.FixMethodOrder;
@@ -281,6 +286,66 @@ public class TestHbaseIndex {
Mockito.verify(table, atMost(numberOfDataFileIds)).put((List<Put>) anyObject());
}
@Test
public void testPutBatchSizeCalculation() {
HbasePutBatchSizeCalculator batchSizeCalculator = new HbasePutBatchSizeCalculator();
// All asserts cases below are derived out of the first
// example below, with change in one parameter at a time.
int putBatchSize = batchSizeCalculator.getBatchSize(10, 16667, 1200, 200, 100, 0.1f);
// Expected batchSize is 8 because in that case, total request sent in one second is below
// 8 (batchSize) * 200 (parallelism) * 10 (maxReqsInOneSecond) * 10 (numRegionServers) * 0.1 (qpsFraction)) => 16000
// We assume requests get distributed to Region Servers uniformly, so each RS gets 1600 request
// 1600 happens to be 10% of 16667 (maxQPSPerRegionServer) as expected.
Assert.assertEquals(putBatchSize, 8);
// Number of Region Servers are halved, total requests sent in a second are also halved, so batchSize is also halved
int putBatchSize2 = batchSizeCalculator.getBatchSize(5, 16667, 1200, 200, 100, 0.1f);
Assert.assertEquals(putBatchSize2, 4);
// If the parallelism is halved, batchSize has to double
int putBatchSize3 = batchSizeCalculator.getBatchSize(10, 16667, 1200, 100, 100, 0.1f);
Assert.assertEquals(putBatchSize3, 16);
// If the parallelism is halved, batchSize has to double.
// This time parallelism is driven by numTasks rather than numExecutors
int putBatchSize4 = batchSizeCalculator.getBatchSize(10, 16667, 100, 200, 100, 0.1f);
Assert.assertEquals(putBatchSize4, 16);
// If sleepTimeMs is halved, batchSize has to halve
int putBatchSize5 = batchSizeCalculator.getBatchSize(10, 16667, 1200, 200, 100, 0.05f);
Assert.assertEquals(putBatchSize5, 4);
// If maxQPSPerRegionServer is doubled, batchSize also doubles
int putBatchSize6 = batchSizeCalculator.getBatchSize(10, 33334, 1200, 200, 100, 0.1f);
Assert.assertEquals(putBatchSize6, 16);
}
@Test
public void testsHBasePutAccessParallelism() {
HoodieWriteConfig config = getConfig();
HBaseIndex index = new HBaseIndex(config);
final JavaRDD<WriteStatus> writeStatusRDD = jsc.parallelize(
Arrays.asList(
getSampleWriteStatus(1, 2),
getSampleWriteStatus(0, 3),
getSampleWriteStatus(10, 0)),
10);
final int hbasePutAccessParallelism = index.getHBasePutAccessParallelism(writeStatusRDD);
Assert.assertEquals(10, writeStatusRDD.getNumPartitions());
Assert.assertEquals(2, hbasePutAccessParallelism);
}
private WriteStatus getSampleWriteStatus(final int numInserts, final int numUpdateWrites) {
final WriteStatus writeStatus = new WriteStatus();
HoodieWriteStat hoodieWriteStat = new HoodieWriteStat();
hoodieWriteStat.setNumInserts(numInserts);
hoodieWriteStat.setNumUpdateWrites(numUpdateWrites);
writeStatus.setStat(hoodieWriteStat);
return writeStatus;
}
private void assertNoWriteErrors(List<WriteStatus> statuses) {
// Verify there are no errors
for (WriteStatus status : statuses) {
@@ -300,8 +365,11 @@ public class TestHbaseIndex {
.withStorageConfig(HoodieStorageConfig.newBuilder().limitFileSize(1024 * 1024).build())
.forTable("test-trip-table").withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.HBASE)
.hbaseZkPort(Integer.valueOf(hbaseConfig.get("hbase.zookeeper.property.clientPort")))
.hbaseZkQuorum(hbaseConfig.get("hbase.zookeeper.quorum")).hbaseTableName(tableName)
.hbaseIndexGetBatchSize(100).hbaseIndexPutBatchSize(100).build());
.withHBaseIndexConfig(
new HoodieHBaseIndexConfig.Builder()
.hbaseZkPort(Integer.valueOf(hbaseConfig.get("hbase.zookeeper.property.clientPort")))
.hbaseZkQuorum(hbaseConfig.get("hbase.zookeeper.quorum")).hbaseTableName(tableName)
.hbaseIndexGetBatchSize(100).build())
.build());
}
}
}

View File

@@ -20,6 +20,7 @@ import static org.junit.Assert.assertTrue;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.config.HoodieHBaseIndexConfig;
import com.uber.hoodie.config.HoodieIndexConfig;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.bloom.HoodieBloomIndex;
@@ -63,7 +64,8 @@ public class TestHoodieIndex {
HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder();
// Different types
HoodieWriteConfig config = clientConfigBuilder.withPath(basePath).withIndexConfig(
indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE).build()).build();
indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE)
.withHBaseIndexConfig(new HoodieHBaseIndexConfig.Builder().build()).build()).build();
assertTrue(HoodieIndex.createIndex(config, jsc) instanceof HBaseIndex);
config = clientConfigBuilder.withPath(basePath)
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();