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 java.io.IOException;
22 import java.util.Random;
23
24 import org.apache.hadoop.hbase.HColumnDescriptor;
25 import org.apache.hadoop.hbase.HTableDescriptor;
26 import org.apache.hadoop.hbase.TableName;
27 import org.apache.hadoop.hbase.client.Admin;
28 import org.apache.hadoop.hbase.io.compress.Compression.Algorithm;
29 import org.apache.hadoop.io.compress.Compressor;
30
31
32
33
34 public class ChangeCompressionAction extends Action {
35 private final TableName tableName;
36
37 private Admin admin;
38 private Random random;
39
40 public ChangeCompressionAction(TableName tableName) {
41 this.tableName = tableName;
42 this.random = new Random();
43 }
44
45 @Override
46 public void init(ActionContext context) throws IOException {
47 super.init(context);
48 this.admin = context.getHBaseIntegrationTestingUtility().getHBaseAdmin();
49 }
50
51 @Override
52 public void perform() throws Exception {
53 HTableDescriptor tableDescriptor = admin.getTableDescriptor(tableName);
54 HColumnDescriptor[] columnDescriptors = tableDescriptor.getColumnFamilies();
55
56 if (columnDescriptors == null || columnDescriptors.length == 0) {
57 return;
58 }
59
60
61
62 Algorithm[] possibleAlgos = Algorithm.values();
63
64
65
66
67
68
69
70 Algorithm algo;
71 do {
72 algo = possibleAlgos[random.nextInt(possibleAlgos.length)];
73
74 try {
75 Compressor c = algo.getCompressor();
76
77
78 algo.returnCompressor(c);
79 break;
80 } catch (Throwable t) {
81 LOG.info("Performing action: Changing compression algorithms to " + algo +
82 " is not supported, pick another one");
83 }
84 } while (true);
85
86 LOG.debug("Performing action: Changing compression algorithms on "
87 + tableName.getNameAsString() + " to " + algo);
88 for (HColumnDescriptor descriptor : columnDescriptors) {
89 if (random.nextBoolean()) {
90 descriptor.setCompactionCompressionType(algo);
91 } else {
92 descriptor.setCompressionType(algo);
93 }
94 }
95
96
97 if (context.isStopping()) {
98 return;
99 }
100
101 admin.modifyTable(tableName, tableDescriptor);
102 }
103 }