1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.client;
19
20 import org.apache.hadoop.conf.Configuration;
21 import org.apache.hadoop.hbase.ServerName;
22 import org.apache.hadoop.hbase.client.backoff.ExponentialClientBackoffPolicy;
23 import org.apache.hadoop.hbase.client.backoff.ServerStatistics;
24 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
25 import org.apache.hadoop.hbase.testclassification.SmallTests;
26 import org.apache.hadoop.hbase.util.Bytes;
27 import org.junit.Test;
28 import org.junit.experimental.categories.Category;
29 import org.mockito.Mockito;
30
31 import static org.junit.Assert.assertEquals;
32 import static org.junit.Assert.assertTrue;
33
34 @Category(SmallTests.class)
35 public class TestClientExponentialBackoff {
36
37 ServerName server = Mockito.mock(ServerName.class);
38 byte[] regionname = Bytes.toBytes("region");
39
40 @Test
41 public void testNulls() {
42 Configuration conf = new Configuration(false);
43 ExponentialClientBackoffPolicy backoff = new ExponentialClientBackoffPolicy(conf);
44 assertEquals(0, backoff.getBackoffTime(null, null, null));
45
46
47 assertEquals(0, backoff.getBackoffTime(server, null, null));
48 assertEquals(0, backoff.getBackoffTime(server, regionname, null));
49
50
51 ServerStatistics stats = new ServerStatistics();
52 assertEquals(0, backoff.getBackoffTime(server, regionname, stats));
53 }
54
55 @Test
56 public void testMaxLoad() {
57 Configuration conf = new Configuration(false);
58 ExponentialClientBackoffPolicy backoff = new ExponentialClientBackoffPolicy(conf);
59
60 ServerStatistics stats = new ServerStatistics();
61 update(stats, 100);
62 assertEquals(ExponentialClientBackoffPolicy.DEFAULT_MAX_BACKOFF, backoff.getBackoffTime(server,
63 regionname, stats));
64
65
66 long max = 100;
67 conf.setLong(ExponentialClientBackoffPolicy.MAX_BACKOFF_KEY, max);
68 ExponentialClientBackoffPolicy backoffShortTimeout = new ExponentialClientBackoffPolicy(conf);
69 assertEquals(max, backoffShortTimeout.getBackoffTime(server, regionname, stats));
70
71
72 update(stats, 101);
73 assertEquals(ExponentialClientBackoffPolicy.DEFAULT_MAX_BACKOFF, backoff.getBackoffTime(server,
74 regionname, stats));
75 assertEquals(max, backoffShortTimeout.getBackoffTime(server, regionname, stats));
76
77
78 update(stats, 99);
79 assertTrue(backoff.getBackoffTime(server,
80 regionname, stats) < ExponentialClientBackoffPolicy.DEFAULT_MAX_BACKOFF);
81 assertTrue(backoffShortTimeout.getBackoffTime(server, regionname, stats) < max);
82 }
83
84
85
86
87
88 @Test
89 public void testResultOrdering() {
90 Configuration conf = new Configuration(false);
91
92 conf.setLong(ExponentialClientBackoffPolicy.MAX_BACKOFF_KEY, Integer.MAX_VALUE);
93 ExponentialClientBackoffPolicy backoff = new ExponentialClientBackoffPolicy(conf);
94
95 ServerStatistics stats = new ServerStatistics();
96 long previous = backoff.getBackoffTime(server, regionname, stats);
97 for (int i = 1; i <= 100; i++) {
98 update(stats, i);
99 long next = backoff.getBackoffTime(server, regionname, stats);
100 assertTrue(
101 "Previous backoff time" + previous + " >= " + next + ", the next backoff time for " +
102 "load " + i, previous < next);
103 previous = next;
104 }
105 }
106
107 @Test
108 public void testHeapOccupancyPolicy() {
109 Configuration conf = new Configuration(false);
110 ExponentialClientBackoffPolicy backoff = new ExponentialClientBackoffPolicy(conf);
111
112 ServerStatistics stats = new ServerStatistics();
113 long backoffTime;
114
115 update(stats, 0, 95);
116 backoffTime = backoff.getBackoffTime(server, regionname, stats);
117 assertTrue("Heap occupancy at low watermark had no effect", backoffTime > 0);
118
119 long previous = backoffTime;
120 update(stats, 0, 96);
121 backoffTime = backoff.getBackoffTime(server, regionname, stats);
122 assertTrue("Increase above low watermark should have increased backoff",
123 backoffTime > previous);
124
125 update(stats, 0, 98);
126 backoffTime = backoff.getBackoffTime(server, regionname, stats);
127 assertEquals("We should be using max backoff when at high watermark", backoffTime,
128 ExponentialClientBackoffPolicy.DEFAULT_MAX_BACKOFF);
129 }
130
131 private void update(ServerStatistics stats, int load) {
132 ClientProtos.RegionLoadStats stat = ClientProtos.RegionLoadStats.newBuilder()
133 .setMemstoreLoad
134 (load).build();
135 stats.update(regionname, stat);
136 }
137
138 private void update(ServerStatistics stats, int memstoreLoad, int heapOccupancy) {
139 ClientProtos.RegionLoadStats stat = ClientProtos.RegionLoadStats.newBuilder()
140 .setMemstoreLoad(memstoreLoad)
141 .setHeapOccupancy(heapOccupancy)
142 .build();
143 stats.update(regionname, stat);
144 }
145 }