1
2
3
4
5
6
7
8
9
10
11
12 package org.apache.hadoop.hbase.quotas;
13
14 import java.util.HashMap;
15 import java.util.HashSet;
16 import java.util.Map;
17 import java.util.Set;
18
19 import org.apache.hadoop.hbase.TableName;
20 import org.apache.hadoop.hbase.classification.InterfaceAudience;
21 import org.apache.hadoop.hbase.classification.InterfaceStability;
22 import org.apache.hadoop.hbase.protobuf.generated.QuotaProtos.Quotas;
23 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
24
25
26
27
28 @InterfaceAudience.Private
29 @InterfaceStability.Evolving
30 public class UserQuotaState extends QuotaState {
31 private Map<String, QuotaLimiter> namespaceLimiters = null;
32 private Map<TableName, QuotaLimiter> tableLimiters = null;
33 private boolean bypassGlobals = false;
34
35 public UserQuotaState() {
36 super();
37 }
38
39 public UserQuotaState(final long updateTs) {
40 super(updateTs);
41 }
42
43 @Override
44 public synchronized String toString() {
45 StringBuilder builder = new StringBuilder();
46 builder.append("UserQuotaState(ts=" + getLastUpdate());
47 if (bypassGlobals) builder.append(" bypass-globals");
48
49 if (isBypass()) {
50 builder.append(" bypass");
51 } else {
52 if (getGlobalLimiterWithoutUpdatingLastQuery() != NoopQuotaLimiter.get()) {
53 builder.append(" global-limiter");
54 }
55
56 if (tableLimiters != null && !tableLimiters.isEmpty()) {
57 builder.append(" [");
58 for (TableName table : tableLimiters.keySet()) {
59 builder.append(" " + table);
60 }
61 builder.append(" ]");
62 }
63
64 if (namespaceLimiters != null && !namespaceLimiters.isEmpty()) {
65 builder.append(" [");
66 for (String ns : namespaceLimiters.keySet()) {
67 builder.append(" " + ns);
68 }
69 builder.append(" ]");
70 }
71 }
72 builder.append(')');
73 return builder.toString();
74 }
75
76
77
78
79 @Override
80 public synchronized boolean isBypass() {
81 return !bypassGlobals && getGlobalLimiterWithoutUpdatingLastQuery() == NoopQuotaLimiter.get()
82 && (tableLimiters == null || tableLimiters.isEmpty())
83 && (namespaceLimiters == null || namespaceLimiters.isEmpty());
84 }
85
86 public synchronized boolean hasBypassGlobals() {
87 return bypassGlobals;
88 }
89
90 @Override
91 public synchronized void setQuotas(final Quotas quotas) {
92 super.setQuotas(quotas);
93 bypassGlobals = quotas.getBypassGlobals();
94 }
95
96
97
98
99
100 public synchronized void setQuotas(final TableName table, Quotas quotas) {
101 tableLimiters = setLimiter(tableLimiters, table, quotas);
102 }
103
104
105
106
107
108 public synchronized void setQuotas(final String namespace, Quotas quotas) {
109 namespaceLimiters = setLimiter(namespaceLimiters, namespace, quotas);
110 }
111
112 private <K> Map<K, QuotaLimiter> setLimiter(Map<K, QuotaLimiter> limiters, final K key,
113 final Quotas quotas) {
114 if (limiters == null) {
115 limiters = new HashMap<K, QuotaLimiter>();
116 }
117
118 QuotaLimiter limiter =
119 quotas.hasThrottle() ? QuotaLimiterFactory.fromThrottle(quotas.getThrottle()) : null;
120 if (limiter != null && !limiter.isBypass()) {
121 limiters.put(key, limiter);
122 } else {
123 limiters.remove(key);
124 }
125 return limiters;
126 }
127
128
129
130
131
132 @Override
133 public synchronized void update(final QuotaState other) {
134 super.update(other);
135
136 if (other instanceof UserQuotaState) {
137 UserQuotaState uOther = (UserQuotaState) other;
138 tableLimiters = updateLimiters(tableLimiters, uOther.tableLimiters);
139 namespaceLimiters = updateLimiters(namespaceLimiters, uOther.namespaceLimiters);
140 bypassGlobals = uOther.bypassGlobals;
141 } else {
142 tableLimiters = null;
143 namespaceLimiters = null;
144 bypassGlobals = false;
145 }
146 }
147
148 private static <K> Map<K, QuotaLimiter> updateLimiters(final Map<K, QuotaLimiter> map,
149 final Map<K, QuotaLimiter> otherMap) {
150 if (map == null) {
151 return otherMap;
152 }
153
154 if (otherMap != null) {
155
156 Set<K> toRemove = new HashSet<K>(map.keySet());
157 toRemove.removeAll(otherMap.keySet());
158 map.keySet().removeAll(toRemove);
159
160
161 for (final Map.Entry<K, QuotaLimiter> entry : otherMap.entrySet()) {
162 QuotaLimiter limiter = map.get(entry.getKey());
163 if (limiter == null) {
164 limiter = entry.getValue();
165 } else {
166 limiter = QuotaLimiterFactory.update(limiter, entry.getValue());
167 }
168 map.put(entry.getKey(), limiter);
169 }
170 return map;
171 }
172 return null;
173 }
174
175
176
177
178
179
180
181 public synchronized QuotaLimiter getTableLimiter(final TableName table) {
182 setLastQuery(EnvironmentEdgeManager.currentTime());
183 if (tableLimiters != null) {
184 QuotaLimiter limiter = tableLimiters.get(table);
185 if (limiter != null) return limiter;
186 }
187 if (namespaceLimiters != null) {
188 QuotaLimiter limiter = namespaceLimiters.get(table.getNamespaceAsString());
189 if (limiter != null) return limiter;
190 }
191 return getGlobalLimiterWithoutUpdatingLastQuery();
192 }
193 }