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
019package org.apache.hadoop.mapreduce;
020
021import java.io.DataInput;
022import java.io.DataOutput;
023import java.io.IOException;
024import java.text.NumberFormat;
025import java.util.EnumMap;
026import java.util.HashMap;
027import java.util.Map;
028
029import org.apache.hadoop.classification.InterfaceAudience;
030import org.apache.hadoop.classification.InterfaceStability;
031import org.apache.hadoop.io.WritableUtils;
032
033
034/**
035 * TaskID represents the immutable and unique identifier for 
036 * a Map or Reduce Task. Each TaskID encompasses multiple attempts made to
037 * execute the Map or Reduce Task, each of which are uniquely indentified by
038 * their TaskAttemptID.
039 * 
040 * TaskID consists of 3 parts. First part is the {@link JobID}, that this 
041 * TaskInProgress belongs to. Second part of the TaskID is either 'm' or 'r' 
042 * representing whether the task is a map task or a reduce task. 
043 * And the third part is the task number. <br> 
044 * An example TaskID is : 
045 * <code>task_200707121733_0003_m_000005</code> , which represents the
046 * fifth map task in the third job running at the jobtracker 
047 * started at <code>200707121733</code>. 
048 * <p>
049 * Applications should never construct or parse TaskID strings
050 * , but rather use appropriate constructors or {@link #forName(String)} 
051 * method. 
052 * 
053 * @see JobID
054 * @see TaskAttemptID
055 */
056@InterfaceAudience.Public
057@InterfaceStability.Stable
058public class TaskID extends org.apache.hadoop.mapred.ID {
059  protected static final String TASK = "task";
060  protected static final NumberFormat idFormat = NumberFormat.getInstance();
061  static {
062    idFormat.setGroupingUsed(false);
063    idFormat.setMinimumIntegerDigits(6);
064  }
065  
066  private JobID jobId;
067  private TaskType type;
068  
069  /**
070   * Constructs a TaskID object from given {@link JobID}.  
071   * @param jobId JobID that this tip belongs to 
072   * @param type the {@link TaskType} of the task 
073   * @param id the tip number
074   */
075  public TaskID(JobID jobId, TaskType type, int id) {
076    super(id);
077    if(jobId == null) {
078      throw new IllegalArgumentException("jobId cannot be null");
079    }
080    this.jobId = jobId;
081    this.type = type;
082  }
083  
084  /**
085   * Constructs a TaskInProgressId object from given parts.
086   * @param jtIdentifier jobTracker identifier
087   * @param jobId job number 
088   * @param type the TaskType 
089   * @param id the tip number
090   */
091  public TaskID(String jtIdentifier, int jobId, TaskType type, int id) {
092    this(new JobID(jtIdentifier, jobId), type, id);
093  }
094
095  /**
096   * Constructs a TaskID object from given {@link JobID}.
097   * @param jobId JobID that this tip belongs to
098   * @param isMap whether the tip is a map
099   * @param id the tip number
100   */
101  @Deprecated
102  public TaskID(JobID jobId, boolean isMap, int id) {
103    this(jobId, isMap ? TaskType.MAP : TaskType.REDUCE, id);
104  }
105
106  /**
107   * Constructs a TaskInProgressId object from given parts.
108   * @param jtIdentifier jobTracker identifier
109   * @param jobId job number
110   * @param isMap whether the tip is a map
111   * @param id the tip number
112   */
113  @Deprecated
114  public TaskID(String jtIdentifier, int jobId, boolean isMap, int id) {
115    this(new JobID(jtIdentifier, jobId), isMap, id);
116  }
117  
118  public TaskID() { 
119    jobId = new JobID();
120  }
121  
122  /** Returns the {@link JobID} object that this tip belongs to */
123  public JobID getJobID() {
124    return jobId;
125  }
126  
127  /**Returns whether this TaskID is a map ID */
128  @Deprecated
129  public boolean isMap() {
130    return type == TaskType.MAP;
131  }
132    
133  /**
134   * Get the type of the task
135   */
136  public TaskType getTaskType() {
137    return type;
138  }
139  
140  @Override
141  public boolean equals(Object o) {
142    if (!super.equals(o))
143      return false;
144
145    TaskID that = (TaskID)o;
146    return this.type == that.type && this.jobId.equals(that.jobId);
147  }
148
149  /**Compare TaskInProgressIds by first jobIds, then by tip numbers. Reduces are 
150   * defined as greater then maps.*/
151  @Override
152  public int compareTo(ID o) {
153    TaskID that = (TaskID)o;
154    int jobComp = this.jobId.compareTo(that.jobId);
155    if(jobComp == 0) {
156      if(this.type == that.type) {
157        return this.id - that.id;
158      }
159      else {
160        return this.type.compareTo(that.type);
161      }
162    }
163    else return jobComp;
164  }
165  @Override
166  public String toString() { 
167    return appendTo(new StringBuilder(TASK)).toString();
168  }
169
170  /**
171   * Add the unique string to the given builder.
172   * @param builder the builder to append to
173   * @return the builder that was passed in
174   */
175  protected StringBuilder appendTo(StringBuilder builder) {
176    return jobId.appendTo(builder).
177                 append(SEPARATOR).
178                 append(CharTaskTypeMaps.getRepresentingCharacter(type)).
179                 append(SEPARATOR).
180                 append(idFormat.format(id));
181  }
182  
183  @Override
184  public int hashCode() {
185    return jobId.hashCode() * 524287 + id;
186  }
187  
188  @Override
189  public void readFields(DataInput in) throws IOException {
190    super.readFields(in);
191    jobId.readFields(in);
192    type = WritableUtils.readEnum(in, TaskType.class);
193  }
194
195  @Override
196  public void write(DataOutput out) throws IOException {
197    super.write(out);
198    jobId.write(out);
199    WritableUtils.writeEnum(out, type);
200  }
201  
202  /** Construct a TaskID object from given string 
203   * @return constructed TaskID object or null if the given String is null
204   * @throws IllegalArgumentException if the given string is malformed
205   */
206  public static TaskID forName(String str) 
207    throws IllegalArgumentException {
208    if(str == null)
209      return null;
210    String exceptionMsg = null;
211    try {
212      String[] parts = str.split("_");
213      if(parts.length == 5) {
214        if(parts[0].equals(TASK)) {
215          String type = parts[3];
216          TaskType t = CharTaskTypeMaps.getTaskType(type.charAt(0));
217          if(t != null) {
218          
219            return new org.apache.hadoop.mapred.TaskID(parts[1], 
220                                                     Integer.parseInt(parts[2]),
221                                                     t, 
222                                                     Integer.parseInt(parts[4]));
223          } else
224            exceptionMsg = "Bad TaskType identifier. TaskId string : " + str
225                + " is not properly formed.";
226        }
227      }
228    }catch (Exception ex) {//fall below
229    }
230    if (exceptionMsg == null) {
231      exceptionMsg = "TaskId string : " + str + " is not properly formed";
232    }
233    throw new IllegalArgumentException(exceptionMsg);
234  }
235  /**
236   * Gets the character representing the {@link TaskType}
237   * @param type the TaskType
238   * @return the character
239   */
240  public static char getRepresentingCharacter(TaskType type) {
241    return CharTaskTypeMaps.getRepresentingCharacter(type);
242  }
243  /**
244   * Gets the {@link TaskType} corresponding to the character
245   * @param c the character
246   * @return the TaskType
247   */
248  public static TaskType getTaskType(char c) {
249    return CharTaskTypeMaps.getTaskType(c);
250  }
251  
252  public static String getAllTaskTypes() {
253    return CharTaskTypeMaps.allTaskTypes;
254  }
255
256  /**
257   * Maintains the mapping from the character representation of a task type to 
258   * the enum class TaskType constants
259   */
260  static class CharTaskTypeMaps {
261    private static EnumMap<TaskType, Character> typeToCharMap = 
262      new EnumMap<TaskType,Character>(TaskType.class);
263    private static Map<Character, TaskType> charToTypeMap = 
264      new HashMap<Character, TaskType>();
265    static String allTaskTypes = "(m|r|s|c|t)";
266    static {
267      setupTaskTypeToCharMapping();
268      setupCharToTaskTypeMapping();
269    }
270    
271    private static void setupTaskTypeToCharMapping() {
272      typeToCharMap.put(TaskType.MAP, 'm');
273      typeToCharMap.put(TaskType.REDUCE, 'r');
274      typeToCharMap.put(TaskType.JOB_SETUP, 's');
275      typeToCharMap.put(TaskType.JOB_CLEANUP, 'c');
276      typeToCharMap.put(TaskType.TASK_CLEANUP, 't');
277    }
278
279    private static void setupCharToTaskTypeMapping() {
280      charToTypeMap.put('m', TaskType.MAP);
281      charToTypeMap.put('r', TaskType.REDUCE);
282      charToTypeMap.put('s', TaskType.JOB_SETUP);
283      charToTypeMap.put('c', TaskType.JOB_CLEANUP);
284      charToTypeMap.put('t', TaskType.TASK_CLEANUP);
285    }
286
287    static char getRepresentingCharacter(TaskType type) {
288      return typeToCharMap.get(type);
289    }
290    static TaskType getTaskType(char c) {
291      return charToTypeMap.get(c);
292    }
293  }
294
295}