1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.chaos.actions;
20
21 import org.apache.hadoop.hbase.HBaseTestingUtility;
22 import org.apache.hadoop.hbase.HConstants;
23 import org.apache.hadoop.hbase.HTableDescriptor;
24 import org.apache.hadoop.hbase.TableName;
25 import org.apache.hadoop.hbase.client.Admin;
26
27 import java.util.Random;
28 import org.slf4j.Logger;
29 import org.slf4j.LoggerFactory;
30
31 public class DecreaseMaxHFileSizeAction extends Action {
32 private static final Logger LOG = LoggerFactory.getLogger(DecreaseMaxHFileSizeAction.class);
33
34 private static final long minFileSize = 1024 * 1024 * 1024L;
35
36 private final long sleepTime;
37 private final TableName tableName;
38 private final Random random;
39
40 public DecreaseMaxHFileSizeAction(long sleepTime, TableName tableName) {
41 this.sleepTime = sleepTime;
42 this.tableName = tableName;
43 this.random = new Random();
44 }
45
46 @Override protected Logger getLogger() {
47 return LOG;
48 }
49
50 @Override
51 public void perform() throws Exception {
52 HBaseTestingUtility util = context.getHBaseIntegrationTestingUtility();
53 Admin admin = util.getHBaseAdmin();
54 HTableDescriptor htd = admin.getTableDescriptor(tableName);
55
56
57 long currentValue = htd.getMaxFileSize();
58
59
60
61
62 if (currentValue <= 0) {
63 currentValue =
64 context.getHBaseCluster().getConf().getLong(HConstants.HREGION_MAX_FILESIZE,
65 HConstants.DEFAULT_MAX_FILE_SIZE);
66 }
67
68
69 long newValue = (long) (currentValue * 0.9);
70
71
72
73 newValue = Math.max(minFileSize, newValue) - (512 - random.nextInt(1024));
74
75
76 htd.setMaxFileSize(newValue);
77
78
79 if (context.isStopping()) {
80 return;
81 }
82
83
84 admin.modifyTable(tableName, htd);
85
86
87 if (sleepTime > 0) {
88 Thread.sleep(sleepTime);
89 }
90 }
91 }