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 */ 018 019 package org.apache.hadoop.lib.service.instrumentation; 020 021 import org.apache.hadoop.lib.server.BaseService; 022 import org.apache.hadoop.lib.server.ServiceException; 023 import org.apache.hadoop.lib.service.Instrumentation; 024 import org.apache.hadoop.lib.service.Scheduler; 025 import org.json.simple.JSONAware; 026 import org.json.simple.JSONObject; 027 import org.json.simple.JSONStreamAware; 028 029 import java.io.IOException; 030 import java.io.Writer; 031 import java.util.ArrayList; 032 import java.util.LinkedHashMap; 033 import java.util.List; 034 import java.util.Map; 035 import java.util.concurrent.ConcurrentHashMap; 036 import java.util.concurrent.TimeUnit; 037 import java.util.concurrent.atomic.AtomicLong; 038 import java.util.concurrent.locks.Lock; 039 import java.util.concurrent.locks.ReentrantLock; 040 041 public class InstrumentationService extends BaseService implements Instrumentation { 042 public static final String PREFIX = "instrumentation"; 043 public static final String CONF_TIMERS_SIZE = "timers.size"; 044 045 private int timersSize; 046 private Lock counterLock; 047 private Lock timerLock; 048 private Lock variableLock; 049 private Lock samplerLock; 050 private Map<String, Map<String, AtomicLong>> counters; 051 private Map<String, Map<String, Timer>> timers; 052 private Map<String, Map<String, VariableHolder>> variables; 053 private Map<String, Map<String, Sampler>> samplers; 054 private List<Sampler> samplersList; 055 private Map<String, Map<String, ?>> all; 056 057 public InstrumentationService() { 058 super(PREFIX); 059 } 060 061 @Override 062 @SuppressWarnings("unchecked") 063 public void init() throws ServiceException { 064 timersSize = getServiceConfig().getInt(CONF_TIMERS_SIZE, 10); 065 counterLock = new ReentrantLock(); 066 timerLock = new ReentrantLock(); 067 variableLock = new ReentrantLock(); 068 samplerLock = new ReentrantLock(); 069 Map<String, VariableHolder> jvmVariables = new ConcurrentHashMap<String, VariableHolder>(); 070 counters = new ConcurrentHashMap<String, Map<String, AtomicLong>>(); 071 timers = new ConcurrentHashMap<String, Map<String, Timer>>(); 072 variables = new ConcurrentHashMap<String, Map<String, VariableHolder>>(); 073 samplers = new ConcurrentHashMap<String, Map<String, Sampler>>(); 074 samplersList = new ArrayList<Sampler>(); 075 all = new LinkedHashMap<String, Map<String, ?>>(); 076 all.put("os-env", System.getenv()); 077 all.put("sys-props", (Map<String, ?>) (Map) System.getProperties()); 078 all.put("jvm", jvmVariables); 079 all.put("counters", (Map) counters); 080 all.put("timers", (Map) timers); 081 all.put("variables", (Map) variables); 082 all.put("samplers", (Map) samplers); 083 084 jvmVariables.put("free.memory", new VariableHolder<Long>(new Instrumentation.Variable<Long>() { 085 public Long getValue() { 086 return Runtime.getRuntime().freeMemory(); 087 } 088 })); 089 jvmVariables.put("max.memory", new VariableHolder<Long>(new Instrumentation.Variable<Long>() { 090 public Long getValue() { 091 return Runtime.getRuntime().maxMemory(); 092 } 093 })); 094 jvmVariables.put("total.memory", new VariableHolder<Long>(new Instrumentation.Variable<Long>() { 095 public Long getValue() { 096 return Runtime.getRuntime().totalMemory(); 097 } 098 })); 099 } 100 101 @Override 102 public void postInit() throws ServiceException { 103 Scheduler scheduler = getServer().get(Scheduler.class); 104 if (scheduler != null) { 105 scheduler.schedule(new SamplersRunnable(), 0, 1, TimeUnit.SECONDS); 106 } 107 } 108 109 @Override 110 public Class getInterface() { 111 return Instrumentation.class; 112 } 113 114 @SuppressWarnings("unchecked") 115 private <T> T getToAdd(String group, String name, Class<T> klass, Lock lock, Map<String, Map<String, T>> map) { 116 boolean locked = false; 117 try { 118 Map<String, T> groupMap = map.get(group); 119 if (groupMap == null) { 120 lock.lock(); 121 locked = true; 122 groupMap = map.get(group); 123 if (groupMap == null) { 124 groupMap = new ConcurrentHashMap<String, T>(); 125 map.put(group, groupMap); 126 } 127 } 128 T element = groupMap.get(name); 129 if (element == null) { 130 if (!locked) { 131 lock.lock(); 132 locked = true; 133 } 134 element = groupMap.get(name); 135 if (element == null) { 136 try { 137 if (klass == Timer.class) { 138 element = (T) new Timer(timersSize); 139 } else { 140 element = klass.newInstance(); 141 } 142 } catch (Exception ex) { 143 throw new RuntimeException(ex); 144 } 145 groupMap.put(name, element); 146 } 147 } 148 return element; 149 } finally { 150 if (locked) { 151 lock.unlock(); 152 } 153 } 154 } 155 156 static class Cron implements Instrumentation.Cron { 157 long start; 158 long lapStart; 159 long own; 160 long total; 161 162 public Cron start() { 163 if (total != 0) { 164 throw new IllegalStateException("Cron already used"); 165 } 166 if (start == 0) { 167 start = System.currentTimeMillis(); 168 lapStart = start; 169 } else if (lapStart == 0) { 170 lapStart = System.currentTimeMillis(); 171 } 172 return this; 173 } 174 175 public Cron stop() { 176 if (total != 0) { 177 throw new IllegalStateException("Cron already used"); 178 } 179 if (lapStart > 0) { 180 own += System.currentTimeMillis() - lapStart; 181 lapStart = 0; 182 } 183 return this; 184 } 185 186 void end() { 187 stop(); 188 total = System.currentTimeMillis() - start; 189 } 190 191 } 192 193 static class Timer implements JSONAware, JSONStreamAware { 194 static final int LAST_TOTAL = 0; 195 static final int LAST_OWN = 1; 196 static final int AVG_TOTAL = 2; 197 static final int AVG_OWN = 3; 198 199 Lock lock = new ReentrantLock(); 200 private long[] own; 201 private long[] total; 202 private int last; 203 private boolean full; 204 private int size; 205 206 public Timer(int size) { 207 this.size = size; 208 own = new long[size]; 209 total = new long[size]; 210 for (int i = 0; i < size; i++) { 211 own[i] = -1; 212 total[i] = -1; 213 } 214 last = -1; 215 } 216 217 long[] getValues() { 218 lock.lock(); 219 try { 220 long[] values = new long[4]; 221 values[LAST_TOTAL] = total[last]; 222 values[LAST_OWN] = own[last]; 223 int limit = (full) ? size : (last + 1); 224 for (int i = 0; i < limit; i++) { 225 values[AVG_TOTAL] += total[i]; 226 values[AVG_OWN] += own[i]; 227 } 228 values[AVG_TOTAL] = values[AVG_TOTAL] / limit; 229 values[AVG_OWN] = values[AVG_OWN] / limit; 230 return values; 231 } finally { 232 lock.unlock(); 233 } 234 } 235 236 void addCron(Cron cron) { 237 cron.end(); 238 lock.lock(); 239 try { 240 last = (last + 1) % size; 241 full = full || last == (size - 1); 242 total[last] = cron.total; 243 own[last] = cron.own; 244 } finally { 245 lock.unlock(); 246 } 247 } 248 249 @SuppressWarnings("unchecked") 250 private JSONObject getJSON() { 251 long[] values = getValues(); 252 JSONObject json = new JSONObject(); 253 json.put("lastTotal", values[0]); 254 json.put("lastOwn", values[1]); 255 json.put("avgTotal", values[2]); 256 json.put("avgOwn", values[3]); 257 return json; 258 } 259 260 @Override 261 public String toJSONString() { 262 return getJSON().toJSONString(); 263 } 264 265 @Override 266 public void writeJSONString(Writer out) throws IOException { 267 getJSON().writeJSONString(out); 268 } 269 270 } 271 272 @Override 273 public Cron createCron() { 274 return new Cron(); 275 } 276 277 @Override 278 public void incr(String group, String name, long count) { 279 AtomicLong counter = getToAdd(group, name, AtomicLong.class, counterLock, counters); 280 counter.addAndGet(count); 281 } 282 283 @Override 284 public void addCron(String group, String name, Instrumentation.Cron cron) { 285 Timer timer = getToAdd(group, name, Timer.class, timerLock, timers); 286 timer.addCron((Cron) cron); 287 } 288 289 static class VariableHolder<E> implements JSONAware, JSONStreamAware { 290 Variable<E> var; 291 292 public VariableHolder() { 293 } 294 295 public VariableHolder(Variable<E> var) { 296 this.var = var; 297 } 298 299 @SuppressWarnings("unchecked") 300 private JSONObject getJSON() { 301 JSONObject json = new JSONObject(); 302 json.put("value", var.getValue()); 303 return json; 304 } 305 306 @Override 307 public String toJSONString() { 308 return getJSON().toJSONString(); 309 } 310 311 @Override 312 public void writeJSONString(Writer out) throws IOException { 313 out.write(toJSONString()); 314 } 315 316 } 317 318 @Override 319 public void addVariable(String group, String name, Variable<?> variable) { 320 VariableHolder holder = getToAdd(group, name, VariableHolder.class, variableLock, variables); 321 holder.var = variable; 322 } 323 324 static class Sampler implements JSONAware, JSONStreamAware { 325 Variable<Long> variable; 326 long[] values; 327 private AtomicLong sum; 328 private int last; 329 private boolean full; 330 331 void init(int size, Variable<Long> variable) { 332 this.variable = variable; 333 values = new long[size]; 334 sum = new AtomicLong(); 335 last = 0; 336 } 337 338 void sample() { 339 int index = last; 340 long valueGoingOut = values[last]; 341 full = full || last == (values.length - 1); 342 last = (last + 1) % values.length; 343 values[index] = variable.getValue(); 344 sum.addAndGet(-valueGoingOut + values[index]); 345 } 346 347 double getRate() { 348 return ((double) sum.get()) / ((full) ? values.length : ((last == 0) ? 1 : last)); 349 } 350 351 @SuppressWarnings("unchecked") 352 private JSONObject getJSON() { 353 JSONObject json = new JSONObject(); 354 json.put("sampler", getRate()); 355 json.put("size", (full) ? values.length : last); 356 return json; 357 } 358 359 @Override 360 public String toJSONString() { 361 return getJSON().toJSONString(); 362 } 363 364 @Override 365 public void writeJSONString(Writer out) throws IOException { 366 out.write(toJSONString()); 367 } 368 } 369 370 @Override 371 public void addSampler(String group, String name, int samplingSize, Variable<Long> variable) { 372 Sampler sampler = getToAdd(group, name, Sampler.class, samplerLock, samplers); 373 samplerLock.lock(); 374 try { 375 sampler.init(samplingSize, variable); 376 samplersList.add(sampler); 377 } finally { 378 samplerLock.unlock(); 379 } 380 } 381 382 class SamplersRunnable implements Runnable { 383 384 @Override 385 public void run() { 386 samplerLock.lock(); 387 try { 388 for (Sampler sampler : samplersList) { 389 sampler.sample(); 390 } 391 } finally { 392 samplerLock.unlock(); 393 } 394 } 395 } 396 397 @Override 398 public Map<String, Map<String, ?>> getSnapshot() { 399 return all; 400 } 401 402 403 }