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.Set;
024import java.util.Stack;
025import java.util.concurrent.ConcurrentHashMap;
026
027import com.google.common.base.Preconditions;
028import com.google.common.collect.Lists;
029import org.apache.hadoop.classification.InterfaceAudience;
030import org.apache.hadoop.conf.Configuration;
031import org.apache.hadoop.hdfs.DFSConfigKeys;
032import org.apache.hadoop.metrics2.util.Metrics2Util.NameValuePair;
033import org.apache.hadoop.metrics2.util.Metrics2Util.TopN;
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,
177      String user, long delta) {
178    RollingWindow window = getRollingWindow(command, user);
179    window.incAt(time, delta);
180  }
181
182  /**
183   * Take a snapshot of current top users in the past period.
184   *
185   * @param time the current time
186   * @return a TopWindow describing the top users for each metric in the 
187   * window.
188   */
189  public TopWindow snapshot(long time) {
190    TopWindow window = new TopWindow(windowLenMs);
191    Set<String> metricNames = metricMap.keySet();
192    LOG.debug("iterating in reported metrics, size={} values={}",
193        metricNames.size(), metricNames);
194    for (Map.Entry<String, RollingWindowMap> entry : metricMap.entrySet()) {
195      String metricName = entry.getKey();
196      RollingWindowMap rollingWindows = entry.getValue();
197      TopN topN = getTopUsersForMetric(time, metricName, rollingWindows);
198      final int size = topN.size();
199      if (size == 0) {
200        continue;
201      }
202      Op op = new Op(metricName, topN.getTotal());
203      window.addOp(op);
204      // Reverse the users from the TopUsers using a stack, 
205      // since we'd like them sorted in descending rather than ascending order
206      Stack<NameValuePair> reverse = new Stack<NameValuePair>();
207      for (int i = 0; i < size; i++) {
208        reverse.push(topN.poll());
209      }
210      for (int i = 0; i < size; i++) {
211        NameValuePair userEntry = reverse.pop();
212        User user = new User(userEntry.getName(), userEntry.getValue());
213        op.addUser(user);
214      }
215    }
216    return window;
217  }
218
219  /**
220   * Calculates the top N users over a time interval.
221   * 
222   * @param time the current time
223   * @param metricName Name of metric
224   * @return
225   */
226  private TopN getTopUsersForMetric(long time, String metricName, 
227      RollingWindowMap rollingWindows) {
228    TopN topN = new TopN(topUsersCnt);
229    Iterator<Map.Entry<String, RollingWindow>> iterator =
230        rollingWindows.entrySet().iterator();
231    while (iterator.hasNext()) {
232      Map.Entry<String, RollingWindow> entry = iterator.next();
233      String userName = entry.getKey();
234      RollingWindow aWindow = entry.getValue();
235      long windowSum = aWindow.getSum(time);
236      // do the gc here
237      if (windowSum == 0) {
238        LOG.debug("gc window of metric: {} userName: {}",
239            metricName, userName);
240        iterator.remove();
241        continue;
242      }
243      LOG.debug("offer window of metric: {} userName: {} sum: {}",
244          metricName, userName, windowSum);
245      topN.offer(new NameValuePair(userName, windowSum));
246    }
247    LOG.debug("topN users size for command {} is: {}",
248        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}