Author: cutting Date: Thu Sep 13 14:15:50 2007 New Revision: 575455 URL: http://svn.apache.org/viewvc?rev=575455&view=rev Log: HADOOP-1879. Remove some unneeded casts. Contributed by Nilay Vaish.
Modified: lucene/hadoop/trunk/CHANGES.txt lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeInfo.java lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDataset.java lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java lucene/hadoop/trunk/src/java/org/apache/hadoop/filecache/DistributedCache.java lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java lucene/hadoop/trunk/src/java/org/apache/hadoop/io/IntWritable.java lucene/hadoop/trunk/src/java/org/apache/hadoop/io/Text.java lucene/hadoop/trunk/src/java/org/apache/hadoop/io/VIntWritable.java lucene/hadoop/trunk/src/java/org/apache/hadoop/io/WritableUtils.java lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Client.java lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Server.java lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/DefaultJobHistoryParser.java lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/FileInputFormat.java lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobHistory.java lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobInProgress.java lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MergeSorter.java lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTracker.java lucene/hadoop/trunk/src/java/org/apache/hadoop/metrics/ganglia/GangliaContext.java lucene/hadoop/trunk/src/java/org/apache/hadoop/metrics/spi/OutputRecord.java lucene/hadoop/trunk/src/java/org/apache/hadoop/net/DNS.java lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/Rcc.java lucene/hadoop/trunk/src/java/org/apache/hadoop/util/RunJar.java lucene/hadoop/trunk/src/java/org/apache/hadoop/util/StringUtils.java Modified: lucene/hadoop/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/CHANGES.txt (original) +++ lucene/hadoop/trunk/CHANGES.txt Thu Sep 13 14:15:50 2007 @@ -226,6 +226,8 @@ HADOOP-1592. Log error messages to the client console when tasks fail. (Amar Kamat via cutting) + HADOOP-1879. Remove some unneeded casts. (Nilay Vaish via cutting) + Release 0.14.1 - 2007-09-04 Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DFSClient.java Thu Sep 13 14:15:50 2007 @@ -685,7 +685,7 @@ firstChunkOffset + " != " + chunkOffset); } - int chunkLen = (int) in.readInt(); + int chunkLen = in.readInt(); // Sanity check the lengths if ( chunkLen < 0 || chunkLen > bytesPerChecksum || Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DataNode.java Thu Sep 13 14:15:50 2007 @@ -877,7 +877,7 @@ // Read one data chunk in each loop. long offset = lastOffset + lastLen; - int len = (int) in.readInt(); + int len = in.readInt(); if ( len < 0 || len > bytesPerChecksum ) { LOG.warn( "Got wrong length during writeBlock(" + block + ") from " + s.getRemoteSocketAddress() + Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeInfo.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeInfo.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeInfo.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/DatanodeInfo.java Thu Sep 13 14:15:50 2007 @@ -308,7 +308,7 @@ this.xceiverCount = in.readInt(); this.location = Text.readString(in); this.hostName = Text.readString(in); - AdminStates newState = (AdminStates) WritableUtils.readEnum(in, + AdminStates newState = WritableUtils.readEnum(in, AdminStates.class); setAdminState(newState); } Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDataset.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDataset.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDataset.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSDataset.java Thu Sep 13 14:15:50 2007 @@ -558,7 +558,7 @@ // if (ongoingCreates.containsKey(b)) { // check how old is the temp file - wait 1 hour - File tmp = (File)ongoingCreates.get(b); + File tmp = ongoingCreates.get(b); if ((System.currentTimeMillis() - tmp.lastModified()) < 3600 * 1000) { throw new IOException("Block " + b + " has already been started (though not completed), and thus cannot be created."); Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/dfs/FSNamesystem.java Thu Sep 13 14:15:50 2007 @@ -581,7 +581,7 @@ client = host2DataNodeMap.getDatanodeByHost(clientMachine); for (Iterator<LocatedBlock> it = blocks.getLocatedBlocks().iterator(); it.hasNext();) { - LocatedBlock block = (LocatedBlock) it.next(); + LocatedBlock block = it.next(); clusterMap.pseudoSortByDistance(client, (DatanodeDescriptor[])(block.getLocations())); } @@ -1136,7 +1136,7 @@ values.size() + " datanodes."); while (it.hasNext()) { Map.Entry<String,Collection<Block>> entry = it.next(); - String storageId = (String) entry.getKey(); + String storageId = entry.getKey(); DatanodeDescriptor node = datanodeMap.get(storageId); Collection<Block> blklist = entry.getValue(); if (blklist.size() > 0) { @@ -2149,7 +2149,7 @@ +"ask "+nodeID.getName()+" to delete "+b.getBlockName()); } } - return (Block[]) obsolete.toArray(new Block[obsolete.size()]); + return obsolete.toArray(new Block[obsolete.size()]); } /** @@ -2841,8 +2841,7 @@ int i = 0; for (Iterator<Block> it = replicateBlocks.iterator(); it.hasNext(); i++) { Block block = it.next(); - DatanodeDescriptor targets[] = - (DatanodeDescriptor[]) replicateTargetSets.get(i); + DatanodeDescriptor targets[] = replicateTargetSets.get(i); int numCurrentReplica = numCurrentReplicas.get(i).liveReplicas(); int numExpectedReplica = blocksMap.getINode(block).getReplication(); if (numCurrentReplica + targets.length >= numExpectedReplica) { Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/filecache/DistributedCache.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/filecache/DistributedCache.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/filecache/DistributedCache.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/filecache/DistributedCache.java Thu Sep 13 14:15:50 2007 @@ -108,7 +108,7 @@ throws IOException { String cacheId = makeRelative(cache, conf); synchronized (cachedArchives) { - CacheStatus lcacheStatus = (CacheStatus) cachedArchives.get(cacheId); + CacheStatus lcacheStatus = cachedArchives.get(cacheId); if (lcacheStatus == null) return; synchronized (lcacheStatus) { @@ -124,7 +124,7 @@ synchronized (cachedArchives) { for (Iterator it = cachedArchives.keySet().iterator(); it.hasNext();) { String cacheId = (String) it.next(); - CacheStatus lcacheStatus = (CacheStatus) cachedArchives.get(cacheId); + CacheStatus lcacheStatus = cachedArchives.get(cacheId); synchronized (lcacheStatus) { if (lcacheStatus.refcount == 0) { // delete this cache entry Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/FileSystem.java Thu Sep 13 14:15:50 2007 @@ -531,7 +531,7 @@ for(int i=0; i<files.length; i++) { listPaths(results, files[i], filter); } - return (Path[]) results.toArray(new Path[results.size()]); + return results.toArray(new Path[results.size()]); } /** Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/fs/InMemoryFileSystem.java Thu Sep 13 14:15:50 2007 @@ -194,7 +194,7 @@ if (exists(f) && !overwrite) { throw new IOException("File already exists:"+f); } - FileAttributes fAttr =(FileAttributes) tempFileAttribs.remove(getPath(f)); + FileAttributes fAttr = tempFileAttribs.remove(getPath(f)); if (fAttr != null) return create(f, fAttr); return null; Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/IntWritable.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/IntWritable.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/IntWritable.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/IntWritable.java Thu Sep 13 14:15:50 2007 @@ -51,7 +51,7 @@ } public int hashCode() { - return (int)value; + return value; } /** Compares two IntWritables. */ Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/Text.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/Text.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/Text.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/Text.java Thu Sep 13 14:15:50 2007 @@ -522,17 +522,17 @@ bytes.mark(); byte b = bytes.get(); bytes.reset(); - int extraBytesToRead = bytesFromUTF8[(int)(b & 0xFF)]; + int extraBytesToRead = bytesFromUTF8[(b & 0xFF)]; if (extraBytesToRead < 0) return -1; // trailing byte! int ch = 0; switch (extraBytesToRead) { - case 5: ch += (int)(bytes.get() & 0xFF); ch <<= 6; /* remember, illegal UTF-8 */ - case 4: ch += (int)(bytes.get() & 0xFF); ch <<= 6; /* remember, illegal UTF-8 */ - case 3: ch += (int)(bytes.get() & 0xFF); ch <<= 6; - case 2: ch += (int)(bytes.get() & 0xFF); ch <<= 6; - case 1: ch += (int)(bytes.get() & 0xFF); ch <<= 6; - case 0: ch += (int)(bytes.get() & 0xFF); + case 5: ch += (bytes.get() & 0xFF); ch <<= 6; /* remember, illegal UTF-8 */ + case 4: ch += (bytes.get() & 0xFF); ch <<= 6; /* remember, illegal UTF-8 */ + case 3: ch += (bytes.get() & 0xFF); ch <<= 6; + case 2: ch += (bytes.get() & 0xFF); ch <<= 6; + case 1: ch += (bytes.get() & 0xFF); ch <<= 6; + case 0: ch += (bytes.get() & 0xFF); } ch -= offsetsFromUTF8[extraBytesToRead]; Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/VIntWritable.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/VIntWritable.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/VIntWritable.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/VIntWritable.java Thu Sep 13 14:15:50 2007 @@ -55,7 +55,7 @@ } public int hashCode() { - return (int)value; + return value; } /** Compares two VIntWritables. */ Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/io/WritableUtils.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/io/WritableUtils.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/io/WritableUtils.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/io/WritableUtils.java Thu Sep 13 14:15:50 2007 @@ -394,7 +394,7 @@ int total = 0; int cur = 0; - while ((total<len) && ((cur = (int) in.skipBytes(len-total)) > 0)) { + while ((total<len) && ((cur = in.skipBytes(len-total)) > 0)) { total += cur; } Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Client.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Client.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Client.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Client.java Thu Sep 13 14:15:50 2007 @@ -261,7 +261,7 @@ if (LOG.isDebugEnabled()) LOG.debug(getName() + " got value #" + id); - Call call = (Call)calls.remove(id); + Call call = calls.remove(id); boolean isError = in.readBoolean(); // read if error if (isError) { call.setResult(null, WritableUtils.readString(in), @@ -515,7 +515,7 @@ throws IOException { Connection connection; synchronized (connections) { - connection = (Connection)connections.get(address); + connection = connections.get(address); if (connection == null) { connection = new Connection(address); connections.put(address, connection); Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Server.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Server.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Server.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/ipc/Server.java Thu Sep 13 14:15:50 2007 @@ -252,7 +252,7 @@ Connection c; synchronized (connectionList) { try { - c = (Connection)connectionList.get(i); + c = connectionList.get(i); } catch (Exception e) {return;} } if (c.timedOut(currentTime)) { @@ -526,7 +526,7 @@ Call call = new Call(id, param, this); synchronized (callQueue) { if (callQueue.size() >= maxQueueSize) { - Call oldCall = (Call) callQueue.removeFirst(); + Call oldCall = callQueue.removeFirst(); LOG.warn("Call queue overflow discarding oldest call " + oldCall); } callQueue.addLast(call); // queue the call @@ -570,7 +570,7 @@ callQueue.wait(timeout); } if (!running) break; - call = (Call)callQueue.removeFirst(); // pop the queue + call = callQueue.removeFirst(); // pop the queue } // throw the message away if it is too old Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/DefaultJobHistoryParser.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/DefaultJobHistoryParser.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/DefaultJobHistoryParser.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/DefaultJobHistoryParser.java Thu Sep 13 14:15:50 2007 @@ -165,7 +165,7 @@ currentTracker = values.get(Keys.START_TIME); jobTrackerToJobs.put(currentTracker, activeJobs); } else if (recType.equals(JobHistory.RecordTypes.Job)) { - String jobId = (String) values.get(Keys.JOBID); + String jobId = values.get(Keys.JOBID); JobHistory.JobInfo job = activeJobs.get(jobId); if (null == job) { job = new JobHistory.JobInfo(jobId); Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/FileInputFormat.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/FileInputFormat.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/FileInputFormat.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/FileInputFormat.java Thu Sep 13 14:15:50 2007 @@ -95,7 +95,7 @@ } } - return (Path[])result.toArray(new Path[result.size()]); + return result.toArray(new Path[result.size()]); } public void validateInput(JobConf job) throws IOException { Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobHistory.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobHistory.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobHistory.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobHistory.java Thu Sep 13 14:15:50 2007 @@ -369,7 +369,7 @@ } String logFileName = JOBTRACKER_START_TIME + "_" + jobId; - PrintWriter writer = (PrintWriter)openJobs.get(logFileName); + PrintWriter writer = openJobs.get(logFileName); if (null != writer){ JobHistory.log(writer, RecordTypes.Job, @@ -430,7 +430,7 @@ String.valueOf(finishedReduces)}); } String logFileName = JOBTRACKER_START_TIME + "_" + jobid; - PrintWriter writer = (PrintWriter)openJobs.get(logFileName); + PrintWriter writer = openJobs.get(logFileName); if (null != writer){ JobHistory.log(writer, RecordTypes.Job, new Enum[] {Keys.JOBID, Keys.FINISH_TIME, Keys.JOB_STATUS, Keys.FINISHED_MAPS, Keys.FINISHED_REDUCES }, @@ -460,7 +460,7 @@ public static void logStarted(String jobId, String taskId, String taskType, long startTime){ if (!disableHistory){ - PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); + PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); if (null != writer){ JobHistory.log(writer, RecordTypes.Task, new Enum[]{Keys.TASKID, Keys.TASK_TYPE , Keys.START_TIME}, new String[]{taskId, taskType, String.valueOf(startTime)}); @@ -477,7 +477,7 @@ public static void logFinished(String jobId, String taskId, String taskType, long finishTime){ if (!disableHistory){ - PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); + PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); if (null != writer){ JobHistory.log(writer, RecordTypes.Task, new Enum[]{Keys.TASKID, Keys.TASK_TYPE, Keys.TASK_STATUS, Keys.FINISH_TIME}, @@ -495,7 +495,7 @@ */ public static void logFailed(String jobId, String taskId, String taskType, long time, String error){ if (!disableHistory){ - PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); + PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); if (null != writer){ JobHistory.log(writer, RecordTypes.Task, new Enum[]{Keys.TASKID, Keys.TASK_TYPE, Keys.TASK_STATUS, Keys.FINISH_TIME, Keys.ERROR}, @@ -531,7 +531,7 @@ */ public static void logStarted(String jobId, String taskId, String taskAttemptId, long startTime, String hostName){ if (!disableHistory){ - PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); + PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); if (null != writer){ JobHistory.log(writer, RecordTypes.MapAttempt, new Enum[]{ Keys.TASK_TYPE, Keys.TASKID, @@ -551,7 +551,7 @@ */ public static void logFinished(String jobId, String taskId, String taskAttemptId, long finishTime, String hostName){ if (!disableHistory){ - PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); + PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); if (null != writer){ JobHistory.log(writer, RecordTypes.MapAttempt, new Enum[]{ Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, @@ -573,7 +573,7 @@ public static void logFailed(String jobId, String taskId, String taskAttemptId, long timestamp, String hostName, String error){ if (!disableHistory){ - PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); + PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); if (null != writer){ JobHistory.log(writer, RecordTypes.MapAttempt, new Enum[]{Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, @@ -595,7 +595,7 @@ public static void logKilled(String jobId, String taskId, String taskAttemptId, long timestamp, String hostName, String error){ if (!disableHistory){ - PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); + PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); if (null != writer){ JobHistory.log(writer, RecordTypes.MapAttempt, new Enum[]{Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, @@ -622,7 +622,7 @@ public static void logStarted(String jobId, String taskId, String taskAttemptId, long startTime, String hostName){ if (!disableHistory){ - PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); + PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); if (null != writer){ JobHistory.log(writer, RecordTypes.ReduceAttempt, new Enum[]{ Keys.TASK_TYPE, Keys.TASKID, @@ -645,7 +645,7 @@ public static void logFinished(String jobId, String taskId, String taskAttemptId, long shuffleFinished, long sortFinished, long finishTime, String hostName){ if (!disableHistory){ - PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); + PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); if (null != writer){ JobHistory.log(writer, RecordTypes.ReduceAttempt, new Enum[]{ Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, @@ -668,7 +668,7 @@ public static void logFailed(String jobId, String taskId, String taskAttemptId, long timestamp, String hostName, String error){ if (!disableHistory){ - PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); + PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); if (null != writer){ JobHistory.log(writer, RecordTypes.ReduceAttempt, new Enum[]{ Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, @@ -690,7 +690,7 @@ public static void logKilled(String jobId, String taskId, String taskAttemptId, long timestamp, String hostName, String error){ if (!disableHistory){ - PrintWriter writer = (PrintWriter)openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); + PrintWriter writer = openJobs.get(JOBTRACKER_START_TIME + "_" + jobId); if (null != writer){ JobHistory.log(writer, RecordTypes.ReduceAttempt, new Enum[]{ Keys.TASK_TYPE, Keys.TASKID, Keys.TASK_ATTEMPT_ID, Keys.TASK_STATUS, Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobInProgress.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobInProgress.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobInProgress.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobInProgress.java Thu Sep 13 14:15:50 2007 @@ -131,7 +131,7 @@ this.jobtracker = jobtracker; this.status = new JobStatus(jobid, 0.0f, 0.0f, JobStatus.PREP); this.startTime = System.currentTimeMillis(); - this.localFs = (LocalFileSystem)FileSystem.getLocal(default_conf); + this.localFs = FileSystem.getLocal(default_conf); JobConf default_job_conf = new JobConf(default_conf); this.localJobFile = default_job_conf.getLocalPath(JobTracker.SUBDIR @@ -1142,8 +1142,7 @@ if (taskCompletionEvents.size() > fromEventId) { int actualMax = Math.min(maxEvents, (taskCompletionEvents.size() - fromEventId)); - events = (TaskCompletionEvent[])taskCompletionEvents.subList( - fromEventId, actualMax + fromEventId).toArray(events); + events = taskCompletionEvents.subList(fromEventId, actualMax + fromEventId).toArray(events); } return events; } Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/JobTracker.java Thu Sep 13 14:15:50 2007 @@ -650,7 +650,7 @@ Properties p = System.getProperties(); for (Iterator it = p.keySet().iterator(); it.hasNext();) { String key = (String) it.next(); - String val = (String) p.getProperty(key); + String val = p.getProperty(key); LOG.debug("Property '" + key + "' is " + val); } } @@ -1694,8 +1694,7 @@ */ private TaskInProgress getTip(String jobid, String tipid) { JobInProgress job = jobs.get(jobid); - return (job == null ? null - : (TaskInProgress) job.getTaskInProgress(tipid)); + return (job == null ? null : job.getTaskInProgress(tipid)); } /** Mark a Task to be killed */ Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MergeSorter.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MergeSorter.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MergeSorter.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/MergeSorter.java Thu Sep 13 14:15:50 2007 @@ -40,7 +40,7 @@ MergeSort m = new MergeSort(this); int count = super.count; if (count == 0) return null; - int [] pointers = (int[])super.pointers; + int [] pointers = super.pointers; int [] pointersCopy = new int[count]; System.arraycopy(pointers, 0, pointersCopy, 0, count); m.mergeSort(pointers, pointersCopy, 0, count); Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTracker.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTracker.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTracker.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/mapred/TaskTracker.java Thu Sep 13 14:15:50 2007 @@ -549,8 +549,7 @@ int actualMax = Math.min(max, (allMapEvents.size() - fromId)); List <TaskCompletionEvent> eventSublist = allMapEvents.subList(fromId, actualMax + fromId); - mapEvents = - (TaskCompletionEvent[])eventSublist.toArray(mapEvents); + mapEvents = eventSublist.toArray(mapEvents); } } return mapEvents; @@ -1580,9 +1579,9 @@ * Called upon startup by the child process, to fetch Task data. */ public synchronized Task getTask(String taskid) throws IOException { - TaskInProgress tip = (TaskInProgress) tasks.get(taskid); + TaskInProgress tip = tasks.get(taskid); if (tip != null) { - return (Task) tip.getTask(); + return tip.getTask(); } else { return null; } @@ -1594,7 +1593,7 @@ public synchronized boolean statusUpdate(String taskid, TaskStatus taskStatus) throws IOException { - TaskInProgress tip = (TaskInProgress) tasks.get(taskid); + TaskInProgress tip = tasks.get(taskid); if (tip != null) { tip.reportProgress(taskStatus); return true; @@ -1609,7 +1608,7 @@ * diagnostic info */ public synchronized void reportDiagnosticInfo(String taskid, String info) throws IOException { - TaskInProgress tip = (TaskInProgress) tasks.get(taskid); + TaskInProgress tip = tasks.get(taskid); if (tip != null) { tip.reportDiagnosticInfo(info); } else { @@ -1626,7 +1625,7 @@ * The task is done. */ public synchronized void done(String taskid) throws IOException { - TaskInProgress tip = (TaskInProgress) tasks.get(taskid); + TaskInProgress tip = tasks.get(taskid); if (tip != null) { tip.reportDone(); } else { @@ -1685,7 +1684,7 @@ void reportTaskFinished(String taskid) { TaskInProgress tip; synchronized (this) { - tip = (TaskInProgress) tasks.get(taskid); + tip = tasks.get(taskid); } if (tip != null) { tip.taskFinished(); @@ -1703,7 +1702,7 @@ */ public synchronized void mapOutputLost(String taskid, String errorMsg) throws IOException { - TaskInProgress tip = (TaskInProgress) tasks.get(taskid); + TaskInProgress tip = tasks.get(taskid); if (tip != null) { tip.mapOutputLost(errorMsg); } else { Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/metrics/ganglia/GangliaContext.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/metrics/ganglia/GangliaContext.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/metrics/ganglia/GangliaContext.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/metrics/ganglia/GangliaContext.java Thu Sep 13 14:15:50 2007 @@ -119,7 +119,7 @@ // emit each metric in turn for (String metricName : outRec.getMetricNames()) { Object metric = outRec.getMetric(metricName); - String type = (String) typeTable.get(metric.getClass()); + String type = typeTable.get(metric.getClass()); emitMetric(metricName, type, metric.toString()); } @@ -151,7 +151,7 @@ } private String getUnits(String metricName) { - String result = (String) unitsTable.get(metricName); + String result = unitsTable.get(metricName); if (result == null) { result = DEFAULT_UNITS; } @@ -159,7 +159,7 @@ } private int getSlope(String metricName) { - String slopeString = (String) slopeTable.get(metricName); + String slopeString = slopeTable.get(metricName); if (slopeString == null) { slopeString = DEFAULT_SLOPE; } @@ -167,7 +167,7 @@ } private int getTmax(String metricName) { - String tmaxString = (String) tmaxTable.get(metricName); + String tmaxString = tmaxTable.get(metricName); if (tmaxString == null) { return DEFAULT_TMAX; } @@ -177,7 +177,7 @@ } private int getDmax(String metricName) { - String dmaxString = (String) dmaxTable.get(metricName); + String dmaxString = dmaxTable.get(metricName); if (dmaxString == null) { return DEFAULT_DMAX; } Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/metrics/spi/OutputRecord.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/metrics/spi/OutputRecord.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/metrics/spi/OutputRecord.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/metrics/spi/OutputRecord.java Thu Sep 13 14:15:50 2007 @@ -66,7 +66,7 @@ * Returns the metric object which can be a Float, Integer, Short or Byte. */ public Number getMetric(String name) { - return (Number) metricMap.get(name); + return metricMap.get(name); } } Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/net/DNS.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/net/DNS.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/net/DNS.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/net/DNS.java Thu Sep 13 14:15:50 2007 @@ -97,7 +97,7 @@ Enumeration e = netIF.getInetAddresses(); while (e.hasMoreElements()) ips.add(((InetAddress) e.nextElement()).getHostAddress()); - return (String[]) ips.toArray(new String[] {}); + return ips.toArray(new String[] {}); } } catch (SocketException e) { return new String[] { InetAddress.getLocalHost().getHostAddress() }; @@ -146,7 +146,7 @@ if (hosts.size() == 0) return new String[] { InetAddress.getLocalHost().getCanonicalHostName() }; else - return (String[]) hosts.toArray(new String[] {}); + return hosts.toArray(new String[] {}); } /** Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/Rcc.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/Rcc.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/Rcc.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/record/compiler/generated/Rcc.java Thu Sep 13 14:15:50 2007 @@ -89,7 +89,7 @@ } catch (IOException e) { } } catch (FileNotFoundException e) { - System.err.println("File " + (String) recFiles.get(i) + + System.err.println("File " + recFiles.get(i) + " Not found."); return 1; } @@ -486,7 +486,7 @@ return (jj_ntk = jj_nt.kind); } - private java.util.Vector jj_expentries = new java.util.Vector(); + private java.util.Vector<int[]> jj_expentries = new java.util.Vector<int[]>(); private int[] jj_expentry; private int jj_kind = -1; @@ -521,7 +521,7 @@ } int[][] exptokseq = new int[jj_expentries.size()][]; for (int i = 0; i < jj_expentries.size(); i++) { - exptokseq[i] = (int[])jj_expentries.elementAt(i); + exptokseq[i] = jj_expentries.elementAt(i); } return new ParseException(token, exptokseq, tokenImage); } Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/util/RunJar.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/util/RunJar.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/util/RunJar.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/util/RunJar.java Thu Sep 13 14:15:50 2007 @@ -149,7 +149,7 @@ Method main = mainClass.getMethod("main", new Class[] { Array.newInstance(String.class, 0).getClass() }); - String[] newArgs = (String[])Arrays.asList(args) + String[] newArgs = Arrays.asList(args) .subList(firstArg, args.length).toArray(new String[0]); try { main.invoke(null, new Object[] { newArgs }); Modified: lucene/hadoop/trunk/src/java/org/apache/hadoop/util/StringUtils.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/java/org/apache/hadoop/util/StringUtils.java?rev=575455&r1=575454&r2=575455&view=diff ============================================================================== --- lucene/hadoop/trunk/src/java/org/apache/hadoop/util/StringUtils.java (original) +++ lucene/hadoop/trunk/src/java/org/apache/hadoop/util/StringUtils.java Thu Sep 13 14:15:50 2007 @@ -267,7 +267,7 @@ while (tokenizer.hasMoreTokens()) { values.add(tokenizer.nextToken()); } - return (String[])values.toArray(new String[values.size()]); + return values.toArray(new String[values.size()]); } /**