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