source: proiecte/HadoopJUnit/hadoop-0.20.1/src/mapred/org/apache/hadoop/mapreduce/CounterGroup.java @ 120

Last change on this file since 120 was 120, checked in by (none), 14 years ago

Added the mail files for the Hadoop JUNit Project

  • Property svn:executable set to *
File size: 5.5 KB
Line 
1/**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements.  See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership.  The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License.  You may obtain a copy of the License at
9 *
10 *     http://www.apache.org/licenses/LICENSE-2.0
11 *
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
17 */
18
19package org.apache.hadoop.mapreduce;
20
21import java.io.DataInput;
22import java.io.DataOutput;
23import java.io.IOException;
24import java.util.Iterator;
25import java.util.MissingResourceException;
26import java.util.ResourceBundle;
27import java.util.TreeMap;
28
29import org.apache.hadoop.io.Text;
30import org.apache.hadoop.io.Writable;
31import org.apache.hadoop.io.WritableUtils;
32
33/**
34 * A group of {@link Counter}s that logically belong together. Typically,
35 * it is an {@link Enum} subclass and the counters are the values.
36 */
37public class CounterGroup implements Writable, Iterable<Counter> {
38  private String name;
39  private String displayName;
40  private TreeMap<String, Counter> counters = new TreeMap<String, Counter>();
41  // Optional ResourceBundle for localization of group and counter names.
42  private ResourceBundle bundle = null;   
43 
44  /**
45   * Returns the specified resource bundle, or throws an exception.
46   * @throws MissingResourceException if the bundle isn't found
47   */
48  private static ResourceBundle getResourceBundle(String enumClassName) {
49    String bundleName = enumClassName.replace('$','_');
50    return ResourceBundle.getBundle(bundleName);
51  }
52
53  protected CounterGroup(String name) {
54    this.name = name;
55    try {
56      bundle = getResourceBundle(name);
57    }
58    catch (MissingResourceException neverMind) {
59    }
60    displayName = localize("CounterGroupName", name);
61  }
62 
63  protected CounterGroup(String name, String displayName) {
64    this.name = name;
65    this.displayName = displayName;
66  }
67 
68  /**
69   * Get the internal name of the group
70   * @return the internal name
71   */
72  public synchronized String getName() {
73    return name;
74  }
75 
76  /**
77   * Get the display name of the group.
78   * @return the human readable name
79   */
80  public synchronized String getDisplayName() {
81    return displayName;
82  }
83
84  synchronized void addCounter(Counter counter) {
85    counters.put(counter.getName(), counter);
86  }
87
88  /**
89   * Internal to find a counter in a group.
90   * @param counterName the name of the counter
91   * @param displayName the display name of the counter
92   * @return the counter that was found or added
93   */
94  protected Counter findCounter(String counterName, String displayName) {
95    Counter result = counters.get(counterName);
96    if (result == null) {
97      result = new Counter(counterName, displayName);
98      counters.put(counterName, result);
99    }
100    return result;
101  }
102
103  public synchronized Counter findCounter(String counterName) {
104    Counter result = counters.get(counterName);
105    if (result == null) {
106      String displayName = localize(counterName, counterName);
107      result = new Counter(counterName, displayName);
108      counters.put(counterName, result);
109    }
110    return result;
111  }
112 
113  public synchronized Iterator<Counter> iterator() {
114    return counters.values().iterator();
115  }
116
117  public synchronized void write(DataOutput out) throws IOException {
118    Text.writeString(out, displayName);
119    WritableUtils.writeVInt(out, counters.size());
120    for(Counter counter: counters.values()) {
121      counter.write(out);
122    }
123  }
124 
125  public synchronized void readFields(DataInput in) throws IOException {
126    displayName = Text.readString(in);
127    counters.clear();
128    int size = WritableUtils.readVInt(in);
129    for(int i=0; i < size; i++) {
130      Counter counter = new Counter();
131      counter.readFields(in);
132      counters.put(counter.getName(), counter);
133    }
134  }
135
136  /**
137   * Looks up key in the ResourceBundle and returns the corresponding value.
138   * If the bundle or the key doesn't exist, returns the default value.
139   */
140  private String localize(String key, String defaultValue) {
141    String result = defaultValue;
142    if (bundle != null) {
143      try {
144        result = bundle.getString(key);
145      }
146      catch (MissingResourceException mre) {
147      }
148    }
149    return result;
150  }
151
152  /**
153   * Returns the number of counters in this group.
154   */
155  public synchronized int size() {
156    return counters.size();
157  }
158
159  public synchronized boolean equals(Object genericRight) {
160    if (genericRight instanceof CounterGroup) {
161      Iterator<Counter> right = ((CounterGroup) genericRight).counters.
162                                       values().iterator();
163      Iterator<Counter> left = counters.values().iterator();
164      while (left.hasNext()) {
165        if (!right.hasNext() || !left.next().equals(right.next())) {
166          return false;
167        }
168      }
169      return !right.hasNext();
170    }
171    return false;
172  }
173
174  public synchronized int hashCode() {
175    return counters.hashCode();
176  }
177 
178  public synchronized void incrAllCounters(CounterGroup rightGroup) {
179    for(Counter right: rightGroup.counters.values()) {
180      Counter left = findCounter(right.getName(), right.getDisplayName());
181      left.increment(right.getValue());
182    }
183  }
184}
Note: See TracBrowser for help on using the repository browser.