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;
024
025import org.apache.hadoop.classification.InterfaceAudience;
026import org.apache.hadoop.classification.InterfaceStability;
027
028/**
029 * TaskAttemptID represents the immutable and unique identifier for 
030 * a task attempt. Each task attempt is one particular instance of a Map or
031 * Reduce Task identified by its TaskID. 
032 * 
033 * TaskAttemptID consists of 2 parts. First part is the 
034 * {@link TaskID}, that this TaskAttemptID belongs to.
035 * Second part is the task attempt number. <br> 
036 * An example TaskAttemptID is : 
037 * <code>attempt_200707121733_0003_m_000005_0</code> , which represents the
038 * zeroth task attempt for the fifth map task in the third job 
039 * running at the jobtracker started at <code>200707121733</code>.
040 * <p>
041 * Applications should never construct or parse TaskAttemptID strings
042 * , but rather use appropriate constructors or {@link #forName(String)} 
043 * method. 
044 * 
045 * @see JobID
046 * @see TaskID
047 */
048@InterfaceAudience.Public
049@InterfaceStability.Stable
050public class TaskAttemptID extends org.apache.hadoop.mapred.ID {
051  protected static final String ATTEMPT = "attempt";
052  private TaskID taskId;
053  
054  /**
055   * Constructs a TaskAttemptID object from given {@link TaskID}.  
056   * @param taskId TaskID that this task belongs to  
057   * @param id the task attempt number
058   */
059  public TaskAttemptID(TaskID taskId, int id) {
060    super(id);
061    if(taskId == null) {
062      throw new IllegalArgumentException("taskId cannot be null");
063    }
064    this.taskId = taskId;
065  }
066  
067  /**
068   * Constructs a TaskId object from given parts.
069   * @param jtIdentifier jobTracker identifier
070   * @param jobId job number 
071   * @param type the TaskType 
072   * @param taskId taskId number
073   * @param id the task attempt number
074   */
075  public TaskAttemptID(String jtIdentifier, int jobId, TaskType type, 
076                       int taskId, int id) {
077    this(new TaskID(jtIdentifier, jobId, type, taskId), id);
078  }
079  
080  public TaskAttemptID() { 
081    taskId = new TaskID();
082  }
083  
084  /** Returns the {@link JobID} object that this task attempt belongs to */
085  public JobID getJobID() {
086    return taskId.getJobID();
087  }
088  
089  /** Returns the {@link TaskID} object that this task attempt belongs to */
090  public TaskID getTaskID() {
091    return taskId;
092  }
093  
094  /**Returns whether this TaskID is a map ID */
095  @Deprecated
096  public boolean isMap() {
097    return taskId.isMap();
098  }
099    
100  /**Returns the TaskType of the TaskAttemptID */
101  public TaskType getTaskType() {
102    return taskId.getTaskType();
103  }
104  @Override
105  public boolean equals(Object o) {
106    if (!super.equals(o))
107      return false;
108
109    TaskAttemptID that = (TaskAttemptID)o;
110    return this.taskId.equals(that.taskId);
111  }
112  
113  /**
114   * Add the unique string to the StringBuilder
115   * @param builder the builder to append ot
116   * @return the builder that was passed in.
117   */
118  protected StringBuilder appendTo(StringBuilder builder) {
119    return taskId.appendTo(builder).append(SEPARATOR).append(id);
120  }
121  
122  @Override
123  public void readFields(DataInput in) throws IOException {
124    super.readFields(in);
125    taskId.readFields(in);
126  }
127
128  @Override
129  public void write(DataOutput out) throws IOException {
130    super.write(out);
131    taskId.write(out);
132  }
133
134  @Override
135  public int hashCode() {
136    return taskId.hashCode() * 5 + id;
137  }
138  
139  /**Compare TaskIds by first tipIds, then by task numbers. */
140  @Override
141  public int compareTo(ID o) {
142    TaskAttemptID that = (TaskAttemptID)o;
143    int tipComp = this.taskId.compareTo(that.taskId);
144    if(tipComp == 0) {
145      return this.id - that.id;
146    }
147    else return tipComp;
148  }
149  @Override
150  public String toString() { 
151    return appendTo(new StringBuilder(ATTEMPT)).toString();
152  }
153
154  /** Construct a TaskAttemptID object from given string 
155   * @return constructed TaskAttemptID object or null if the given String is null
156   * @throws IllegalArgumentException if the given string is malformed
157   */
158  public static TaskAttemptID forName(String str
159                                      ) throws IllegalArgumentException {
160    if(str == null)
161      return null;
162    String exceptionMsg = null;
163    try {
164      String[] parts = str.split(Character.toString(SEPARATOR));
165      if(parts.length == 6) {
166        if(parts[0].equals(ATTEMPT)) {
167          String type = parts[3];
168          TaskType t = TaskID.getTaskType(type.charAt(0));
169          if(t != null) {
170            return new org.apache.hadoop.mapred.TaskAttemptID
171            (parts[1],
172             Integer.parseInt(parts[2]),
173             t, Integer.parseInt(parts[4]), 
174             Integer.parseInt(parts[5]));  
175          } else
176            exceptionMsg = "Bad TaskType identifier. TaskAttemptId string : "
177                + str + " is not properly formed.";
178        }
179      }
180    } catch (Exception ex) {
181      //fall below
182    }
183    if (exceptionMsg == null) {
184      exceptionMsg = "TaskAttemptId string : " + str
185          + " is not properly formed";
186    }
187    throw new IllegalArgumentException(exceptionMsg);
188  }
189
190}