001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hdfs.server.namenode.top.window; 019 020import java.util.Iterator; 021import java.util.List; 022import java.util.Map; 023import java.util.PriorityQueue; 024import java.util.Set; 025import java.util.Stack; 026import java.util.concurrent.ConcurrentHashMap; 027 028import com.google.common.base.Preconditions; 029import com.google.common.collect.Lists; 030import com.google.common.primitives.Ints; 031import org.apache.hadoop.classification.InterfaceAudience; 032import org.apache.hadoop.conf.Configuration; 033import org.apache.hadoop.hdfs.DFSConfigKeys; 034import org.slf4j.Logger; 035import org.slf4j.LoggerFactory; 036 037/** 038 * A class to manage the set of {@link RollingWindow}s. This class is the 039 * interface of metrics system to the {@link RollingWindow}s to retrieve the 040 * current top metrics. 041 * <p/> 042 * Thread-safety is provided by each {@link RollingWindow} being thread-safe as 043 * well as {@link ConcurrentHashMap} for the collection of them. 044 */ 045@InterfaceAudience.Private 046public class RollingWindowManager { 047 public static final Logger LOG = LoggerFactory.getLogger( 048 RollingWindowManager.class); 049 050 private final int windowLenMs; 051 private final int bucketsPerWindow; // e.g., 10 buckets per minute 052 private final int topUsersCnt; // e.g., report top 10 metrics 053 054 static private class RollingWindowMap extends 055 ConcurrentHashMap<String, RollingWindow> { 056 private static final long serialVersionUID = -6785807073237052051L; 057 } 058 059 /** 060 * Represents a snapshot of the rolling window. It contains one Op per 061 * operation in the window, with ranked users for each Op. 062 */ 063 public static class TopWindow { 064 private final int windowMillis; 065 private final List<Op> top; 066 067 public TopWindow(int windowMillis) { 068 this.windowMillis = windowMillis; 069 this.top = Lists.newArrayList(); 070 } 071 072 public void addOp(Op op) { 073 top.add(op); 074 } 075 076 public int getWindowLenMs() { 077 return windowMillis; 078 } 079 080 public List<Op> getOps() { 081 return top; 082 } 083 } 084 085 /** 086 * Represents an operation within a TopWindow. It contains a ranked 087 * set of the top users for the operation. 088 */ 089 public static class Op { 090 private final String opType; 091 private final List<User> topUsers; 092 private final long totalCount; 093 094 public Op(String opType, long totalCount) { 095 this.opType = opType; 096 this.topUsers = Lists.newArrayList(); 097 this.totalCount = totalCount; 098 } 099 100 public void addUser(User u) { 101 topUsers.add(u); 102 } 103 104 public String getOpType() { 105 return opType; 106 } 107 108 public List<User> getTopUsers() { 109 return topUsers; 110 } 111 112 public long getTotalCount() { 113 return totalCount; 114 } 115 } 116 117 /** 118 * Represents a user who called an Op within a TopWindow. Specifies the 119 * user and the number of times the user called the operation. 120 */ 121 public static class User { 122 private final String user; 123 private final long count; 124 125 public User(String user, long count) { 126 this.user = user; 127 this.count = count; 128 } 129 130 public String getUser() { 131 return user; 132 } 133 134 public long getCount() { 135 return count; 136 } 137 } 138 139 /** 140 * A mapping from each reported metric to its {@link RollingWindowMap} that 141 * maintains the set of {@link RollingWindow}s for the users that have 142 * operated on that metric. 143 */ 144 public ConcurrentHashMap<String, RollingWindowMap> metricMap = 145 new ConcurrentHashMap<String, RollingWindowMap>(); 146 147 public RollingWindowManager(Configuration conf, int reportingPeriodMs) { 148 149 windowLenMs = reportingPeriodMs; 150 bucketsPerWindow = 151 conf.getInt(DFSConfigKeys.NNTOP_BUCKETS_PER_WINDOW_KEY, 152 DFSConfigKeys.NNTOP_BUCKETS_PER_WINDOW_DEFAULT); 153 Preconditions.checkArgument(bucketsPerWindow > 0, 154 "a window should have at least one bucket"); 155 Preconditions.checkArgument(bucketsPerWindow <= windowLenMs, 156 "the minimum size of a bucket is 1 ms"); 157 //same-size buckets 158 Preconditions.checkArgument(windowLenMs % bucketsPerWindow == 0, 159 "window size must be a multiplication of number of buckets"); 160 topUsersCnt = 161 conf.getInt(DFSConfigKeys.NNTOP_NUM_USERS_KEY, 162 DFSConfigKeys.NNTOP_NUM_USERS_DEFAULT); 163 Preconditions.checkArgument(topUsersCnt > 0, 164 "the number of requested top users must be at least 1"); 165 } 166 167 /** 168 * Called when the metric command is changed by "delta" units at time "time" 169 * via user "user" 170 * 171 * @param time the time of the event 172 * @param command the metric that is updated, e.g., the operation name 173 * @param user the user that updated the metric 174 * @param delta the amount of change in the metric, e.g., +1 175 */ 176 public void recordMetric(long time, String command, String user, long delta) { 177 RollingWindow window = getRollingWindow(command, user); 178 window.incAt(time, delta); 179 } 180 181 /** 182 * Take a snapshot of current top users in the past period. 183 * 184 * @param time the current time 185 * @return a TopWindow describing the top users for each metric in the 186 * window. 187 */ 188 public TopWindow snapshot(long time) { 189 TopWindow window = new TopWindow(windowLenMs); 190 if (LOG.isDebugEnabled()) { 191 Set<String> metricNames = metricMap.keySet(); 192 LOG.debug("iterating in reported metrics, size={} values={}", 193 metricNames.size(), metricNames); 194 } 195 for (Map.Entry<String, RollingWindowMap> entry : metricMap.entrySet()) { 196 String metricName = entry.getKey(); 197 RollingWindowMap rollingWindows = entry.getValue(); 198 TopN topN = getTopUsersForMetric(time, metricName, rollingWindows); 199 final int size = topN.size(); 200 if (size == 0) { 201 continue; 202 } 203 Op op = new Op(metricName, topN.getTotal()); 204 window.addOp(op); 205 // Reverse the users from the TopUsers using a stack, 206 // since we'd like them sorted in descending rather than ascending order 207 Stack<NameValuePair> reverse = new Stack<NameValuePair>(); 208 for (int i = 0; i < size; i++) { 209 reverse.push(topN.poll()); 210 } 211 for (int i = 0; i < size; i++) { 212 NameValuePair userEntry = reverse.pop(); 213 User user = new User(userEntry.name, Long.valueOf(userEntry.value)); 214 op.addUser(user); 215 } 216 } 217 return window; 218 } 219 220 /** 221 * Calculates the top N users over a time interval. 222 * 223 * @param time the current time 224 * @param metricName Name of metric 225 * @return 226 */ 227 private TopN getTopUsersForMetric(long time, String metricName, 228 RollingWindowMap rollingWindows) { 229 TopN topN = new TopN(topUsersCnt); 230 Iterator<Map.Entry<String, RollingWindow>> iterator = 231 rollingWindows.entrySet().iterator(); 232 while (iterator.hasNext()) { 233 Map.Entry<String, RollingWindow> entry = iterator.next(); 234 String userName = entry.getKey(); 235 RollingWindow aWindow = entry.getValue(); 236 long windowSum = aWindow.getSum(time); 237 // do the gc here 238 if (windowSum == 0) { 239 LOG.debug("gc window of metric: {} userName: {}", 240 metricName, userName); 241 iterator.remove(); 242 continue; 243 } 244 LOG.debug("offer window of metric: {} userName: {} sum: {}", 245 metricName, userName, windowSum); 246 topN.offer(new NameValuePair(userName, windowSum)); 247 } 248 LOG.info("topN size for command {} is: {}", metricName, topN.size()); 249 return topN; 250 } 251 252 /** 253 * Get the rolling window specified by metric and user. 254 * 255 * @param metric the updated metric 256 * @param user the user that updated the metric 257 * @return the rolling window 258 */ 259 private RollingWindow getRollingWindow(String metric, String user) { 260 RollingWindowMap rwMap = metricMap.get(metric); 261 if (rwMap == null) { 262 rwMap = new RollingWindowMap(); 263 RollingWindowMap prevRwMap = metricMap.putIfAbsent(metric, rwMap); 264 if (prevRwMap != null) { 265 rwMap = prevRwMap; 266 } 267 } 268 RollingWindow window = rwMap.get(user); 269 if (window != null) { 270 return window; 271 } 272 window = new RollingWindow(windowLenMs, bucketsPerWindow); 273 RollingWindow prevWindow = rwMap.putIfAbsent(user, window); 274 if (prevWindow != null) { 275 window = prevWindow; 276 } 277 return window; 278 } 279 280 /** 281 * A pair of a name and its corresponding value. Defines a custom 282 * comparator so the TopN PriorityQueue sorts based on the count. 283 */ 284 static private class NameValuePair implements Comparable<NameValuePair> { 285 String name; 286 long value; 287 288 public NameValuePair(String metricName, long value) { 289 this.name = metricName; 290 this.value = value; 291 } 292 293 @Override 294 public int compareTo(NameValuePair other) { 295 return (int) (value - other.value); 296 } 297 298 @Override 299 public boolean equals(Object other) { 300 if (other instanceof NameValuePair) { 301 return compareTo((NameValuePair)other) == 0; 302 } 303 return false; 304 } 305 306 @Override 307 public int hashCode() { 308 return Long.valueOf(value).hashCode(); 309 } 310 } 311 312 /** 313 * A fixed-size priority queue, used to retrieve top-n of offered entries. 314 */ 315 static private class TopN extends PriorityQueue<NameValuePair> { 316 private static final long serialVersionUID = 5134028249611535803L; 317 int n; // > 0 318 private long total = 0; 319 320 TopN(int n) { 321 super(n); 322 this.n = n; 323 } 324 325 @Override 326 public boolean offer(NameValuePair entry) { 327 updateTotal(entry.value); 328 if (size() == n) { 329 NameValuePair smallest = peek(); 330 if (smallest.value >= entry.value) { 331 return false; 332 } 333 poll(); // remove smallest 334 } 335 return super.offer(entry); 336 } 337 338 private void updateTotal(long value) { 339 total += value; 340 } 341 342 public long getTotal() { 343 return total; 344 } 345 } 346}