Merge trunk to HDFS-4685.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-4685@1549906 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Chris Nauroth 2013-12-10 17:35:17 +00:00
commit aff5be250a
23 changed files with 511 additions and 42 deletions

View File

@ -175,6 +175,11 @@ public class LineReader implements Closeable {
}
}
protected int fillBuffer(InputStream in, byte[] buffer, boolean inDelimiter)
throws IOException {
return in.read(buffer);
}
/**
* Read a line terminated by one of CR, LF, or CRLF.
*/
@ -208,7 +213,7 @@ public class LineReader implements Closeable {
if (prevCharCR) {
++bytesConsumed; //account for CR from previous read
}
bufferLength = in.read(buffer);
bufferLength = fillBuffer(in, buffer, prevCharCR);
if (bufferLength <= 0) {
break; // EOF
}
@ -296,7 +301,7 @@ public class LineReader implements Closeable {
int startPosn = bufferPosn; // Start from previous end position
if (bufferPosn >= bufferLength) {
startPosn = bufferPosn = 0;
bufferLength = in.read(buffer);
bufferLength = fillBuffer(in, buffer, ambiguousByteCount > 0);
if (bufferLength <= 0) {
str.append(recordDelimiterBytes, 0, ambiguousByteCount);
break; // EOF

View File

@ -587,9 +587,6 @@ Release 2.4.0 - UNRELEASED
HDFS-5633. Improve OfflineImageViewer to use less memory. (jing9)
HDFS-4983. Numeric usernames do not work with WebHDFS FS. (Yongjun Zhang via
jing9)
OPTIMIZATIONS
HDFS-5239. Allow FSNamesystem lock fairness to be configurable (daryn)
@ -693,6 +690,9 @@ Release 2.3.0 - UNRELEASED
HDFS-5568. Support includeSnapshots option with Fsck command. (Vinayakumar B via umamahesh)
HDFS-4983. Numeric usernames do not work with WebHDFS FS. (Yongjun Zhang via
jing9)
OPTIMIZATIONS
BUG FIXES

View File

@ -178,6 +178,9 @@ Release 2.4.0 - UNRELEASED
MAPREDUCE-5481. Enable uber jobs to have multiple reducers (Sandy Ryza)
MAPREDUCE-5052. Job History UI and web services confusing job start time and
job submit time (Chen He via jeagles)
OPTIMIZATIONS
MAPREDUCE-5484. YarnChild unnecessarily loads job conf twice (Sandy Ryza)
@ -231,6 +234,9 @@ Release 2.4.0 - UNRELEASED
MAPREDUCE-5632. TestRMContainerAllocator#testUpdatedNodes fails (jeagles)
MAPREDUCE-5656. bzip2 codec can drop records when reading data in splits
(jlowe)
Release 2.3.0 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -520,6 +520,11 @@ public class JobHistoryEventHandler extends AbstractService
mi.getJobIndexInfo().setSubmitTime(jobSubmittedEvent.getSubmitTime());
mi.getJobIndexInfo().setQueueName(jobSubmittedEvent.getJobQueueName());
}
//initialize the launchTime in the JobIndexInfo of MetaInfo
if(event.getHistoryEvent().getEventType() == EventType.JOB_INITED ){
JobInitedEvent jie = (JobInitedEvent) event.getHistoryEvent();
mi.getJobIndexInfo().setJobStartTime(jie.getLaunchTime());
}
// If this is JobFinishedEvent, close the writer and setup the job-index
if (event.getHistoryEvent().getEventType() == EventType.JOB_FINISHED) {

View File

@ -155,6 +155,7 @@ public class MockJobs extends MockApps {
public static JobReport newJobReport(JobId id) {
JobReport report = Records.newRecord(JobReport.class);
report.setJobId(id);
report.setSubmitTime(System.currentTimeMillis()-DT);
report
.setStartTime(System.currentTimeMillis() - (int) (Math.random() * DT));
report.setFinishTime(System.currentTimeMillis()

View File

@ -51,6 +51,7 @@ public class FileNameIndexUtils {
private static final int NUM_REDUCES_INDEX = 6;
private static final int JOB_STATUS_INDEX = 7;
private static final int QUEUE_NAME_INDEX = 8;
private static final int JOB_START_TIME_INDEX = 9;
/**
* Constructs the job history file name from the JobIndexInfo.
@ -64,7 +65,7 @@ public class FileNameIndexUtils {
sb.append(escapeDelimiters(TypeConverter.fromYarn(indexInfo.getJobId()).toString()));
sb.append(DELIMITER);
//StartTime
//SubmitTime
sb.append(indexInfo.getSubmitTime());
sb.append(DELIMITER);
@ -94,6 +95,10 @@ public class FileNameIndexUtils {
//QueueName
sb.append(indexInfo.getQueueName());
sb.append(DELIMITER);
//JobStartTime
sb.append(indexInfo.getJobStartTime());
sb.append(JobHistoryUtils.JOB_HISTORY_FILE_EXTENSION);
return encodeJobHistoryFileName(sb.toString());
@ -161,6 +166,14 @@ public class FileNameIndexUtils {
indexInfo.setQueueName(
decodeJobHistoryFileName(jobDetails[QUEUE_NAME_INDEX]));
try{
indexInfo.setJobStartTime(
Long.parseLong(decodeJobHistoryFileName(jobDetails[JOB_START_TIME_INDEX])));
} catch (NumberFormatException e){
LOG.warn("Unable to parse launch time from job history file "
+ jhFileName + " : " + e);
}
} catch (IndexOutOfBoundsException e) {
LOG.warn("Parsing job history file with partial data encoded into name: "
+ jhFileName);

View File

@ -34,6 +34,7 @@ public class JobIndexInfo {
private int numMaps;
private int numReduces;
private String jobStatus;
private long jobStartTime;
public JobIndexInfo() {
}
@ -48,6 +49,7 @@ public class JobIndexInfo {
this.numMaps = numMaps;
this.numReduces = numReduces;
this.jobStatus = jobStatus;
this.jobStartTime = -1;
}
public long getSubmitTime() {
@ -104,6 +106,12 @@ public class JobIndexInfo {
public void setJobStatus(String jobStatus) {
this.jobStatus = jobStatus;
}
public long getJobStartTime() {
return jobStartTime;
}
public void setJobStartTime(long lTime) {
this.jobStartTime = lTime;
}
@Override
public String toString() {

View File

@ -48,6 +48,7 @@ public class TestFileNameIndexUtils {
+ FileNameIndexUtils.DELIMITER + "%s"
+ FileNameIndexUtils.DELIMITER + "%s"
+ FileNameIndexUtils.DELIMITER + "%s"
+ FileNameIndexUtils.DELIMITER + "%s"
+ JobHistoryUtils.JOB_HISTORY_FILE_EXTENSION;
private static final String JOB_ID = "job_1317928501754_0001";
@ -67,6 +68,7 @@ public class TestFileNameIndexUtils {
private static final String NUM_REDUCES = "1";
private static final String JOB_STATUS = "SUCCEEDED";
private static final String QUEUE_NAME = "default";
private static final String JOB_START_TIME = "1317928742060";
@Test
public void testEncodingDecodingEquivalence() throws IOException {
@ -82,6 +84,7 @@ public class TestFileNameIndexUtils {
info.setNumReduces(Integer.parseInt(NUM_REDUCES));
info.setJobStatus(JOB_STATUS);
info.setQueueName(QUEUE_NAME);
info.setJobStartTime(Long.parseLong(JOB_START_TIME));
String jobHistoryFile = FileNameIndexUtils.getDoneFileName(info);
JobIndexInfo parsedInfo = FileNameIndexUtils.getIndexInfo(jobHistoryFile);
@ -104,6 +107,8 @@ public class TestFileNameIndexUtils {
info.getJobStatus(), parsedInfo.getJobStatus());
Assert.assertEquals("Queue name different after encoding and decoding",
info.getQueueName(), parsedInfo.getQueueName());
Assert.assertEquals("Job start time different after encoding and decoding",
info.getJobStartTime(), parsedInfo.getJobStartTime());
}
@Test
@ -120,6 +125,7 @@ public class TestFileNameIndexUtils {
info.setNumReduces(Integer.parseInt(NUM_REDUCES));
info.setJobStatus(JOB_STATUS);
info.setQueueName(QUEUE_NAME);
info.setJobStartTime(Long.parseLong(JOB_START_TIME));
String jobHistoryFile = FileNameIndexUtils.getDoneFileName(info);
Assert.assertTrue("User name not encoded correctly into job history file",
@ -137,7 +143,8 @@ public class TestFileNameIndexUtils {
NUM_MAPS,
NUM_REDUCES,
JOB_STATUS,
QUEUE_NAME);
QUEUE_NAME,
JOB_START_TIME);
JobIndexInfo info = FileNameIndexUtils.getIndexInfo(jobHistoryFile);
Assert.assertEquals("User name doesn't match",
@ -158,6 +165,7 @@ public class TestFileNameIndexUtils {
info.setNumReduces(Integer.parseInt(NUM_REDUCES));
info.setJobStatus(JOB_STATUS);
info.setQueueName(QUEUE_NAME);
info.setJobStartTime(Long.parseLong(JOB_START_TIME));
String jobHistoryFile = FileNameIndexUtils.getDoneFileName(info);
Assert.assertTrue("Job name not encoded correctly into job history file",
@ -175,7 +183,8 @@ public class TestFileNameIndexUtils {
NUM_MAPS,
NUM_REDUCES,
JOB_STATUS,
QUEUE_NAME);
QUEUE_NAME,
JOB_START_TIME );
JobIndexInfo info = FileNameIndexUtils.getIndexInfo(jobHistoryFile);
Assert.assertEquals("Job name doesn't match",

View File

@ -36,6 +36,8 @@ import org.apache.hadoop.io.compress.CompressionCodecFactory;
import org.apache.hadoop.io.compress.Decompressor;
import org.apache.hadoop.io.compress.SplitCompressionInputStream;
import org.apache.hadoop.io.compress.SplittableCompressionCodec;
import org.apache.hadoop.mapreduce.lib.input.CompressedSplitLineReader;
import org.apache.hadoop.mapreduce.lib.input.SplitLineReader;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.Log;
@ -52,7 +54,7 @@ public class LineRecordReader implements RecordReader<LongWritable, Text> {
private long start;
private long pos;
private long end;
private LineReader in;
private SplitLineReader in;
private FSDataInputStream fileIn;
private final Seekable filePosition;
int maxLineLength;
@ -111,17 +113,18 @@ public class LineRecordReader implements RecordReader<LongWritable, Text> {
((SplittableCompressionCodec)codec).createInputStream(
fileIn, decompressor, start, end,
SplittableCompressionCodec.READ_MODE.BYBLOCK);
in = new LineReader(cIn, job, recordDelimiter);
in = new CompressedSplitLineReader(cIn, job, recordDelimiter);
start = cIn.getAdjustedStart();
end = cIn.getAdjustedEnd();
filePosition = cIn; // take pos from compressed stream
} else {
in = new LineReader(codec.createInputStream(fileIn, decompressor), job, recordDelimiter);
in = new SplitLineReader(codec.createInputStream(fileIn,
decompressor), job, recordDelimiter);
filePosition = fileIn;
}
} else {
fileIn.seek(start);
in = new LineReader(fileIn, job, recordDelimiter);
in = new SplitLineReader(fileIn, job, recordDelimiter);
filePosition = fileIn;
}
// If this is not the first split, we always throw away first record
@ -141,7 +144,7 @@ public class LineRecordReader implements RecordReader<LongWritable, Text> {
public LineRecordReader(InputStream in, long offset, long endOffset,
int maxLineLength, byte[] recordDelimiter) {
this.maxLineLength = maxLineLength;
this.in = new LineReader(in, recordDelimiter);
this.in = new SplitLineReader(in, recordDelimiter);
this.start = offset;
this.pos = offset;
this.end = endOffset;
@ -159,7 +162,7 @@ public class LineRecordReader implements RecordReader<LongWritable, Text> {
throws IOException{
this.maxLineLength = job.getInt(org.apache.hadoop.mapreduce.lib.input.
LineRecordReader.MAX_LINE_LENGTH, Integer.MAX_VALUE);
this.in = new LineReader(in, job, recordDelimiter);
this.in = new SplitLineReader(in, job, recordDelimiter);
this.start = offset;
this.pos = offset;
this.end = endOffset;
@ -200,7 +203,7 @@ public class LineRecordReader implements RecordReader<LongWritable, Text> {
// We always read one extra line, which lies outside the upper
// split limit i.e. (end - 1)
while (getFilePosition() <= end) {
while (getFilePosition() <= end || in.needAdditionalRecordAfterSplit()) {
key.set(pos);
int newSize = in.readLine(value, maxLineLength,

View File

@ -0,0 +1,168 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.lib.input;
import java.io.IOException;
import java.io.InputStream;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.io.compress.SplitCompressionInputStream;
/**
* Line reader for compressed splits
*
* Reading records from a compressed split is tricky, as the
* LineRecordReader is using the reported compressed input stream
* position directly to determine when a split has ended. In addition the
* compressed input stream is usually faking the actual byte position, often
* updating it only after the first compressed block after the split is
* accessed.
*
* Depending upon where the last compressed block of the split ends relative
* to the record delimiters it can be easy to accidentally drop the last
* record or duplicate the last record between this split and the next.
*
* Split end scenarios:
*
* 1) Last block of split ends in the middle of a record
* Nothing special that needs to be done here, since the compressed input
* stream will report a position after the split end once the record
* is fully read. The consumer of the next split will discard the
* partial record at the start of the split normally, and no data is lost
* or duplicated between the splits.
*
* 2) Last block of split ends in the middle of a delimiter
* The line reader will continue to consume bytes into the next block to
* locate the end of the delimiter. If a custom delimiter is being used
* then the next record must be read by this split or it will be dropped.
* The consumer of the next split will not recognize the partial
* delimiter at the beginning of its split and will discard it along with
* the next record.
*
* However for the default delimiter processing there is a special case
* because CR, LF, and CRLF are all valid record delimiters. If the
* block ends with a CR then the reader must peek at the next byte to see
* if it is an LF and therefore part of the same record delimiter.
* Peeking at the next byte is an access to the next block and triggers
* the stream to report the end of the split. There are two cases based
* on the next byte:
*
* A) The next byte is LF
* The split needs to end after the current record is returned. The
* consumer of the next split will discard the first record, which
* is degenerate since LF is itself a delimiter, and start consuming
* records after that byte. If the current split tries to read
* another record then the record will be duplicated between splits.
*
* B) The next byte is not LF
* The current record will be returned but the stream will report
* the split has ended due to the peek into the next block. If the
* next record is not read then it will be lost, as the consumer of
* the next split will discard it before processing subsequent
* records. Therefore the next record beyond the reported split end
* must be consumed by this split to avoid data loss.
*
* 3) Last block of split ends at the beginning of a delimiter
* This is equivalent to case 1, as the reader will consume bytes into
* the next block and trigger the end of the split. No further records
* should be read as the consumer of the next split will discard the
* (degenerate) record at the beginning of its split.
*
* 4) Last block of split ends at the end of a delimiter
* Nothing special needs to be done here. The reader will not start
* examining the bytes into the next block until the next record is read,
* so the stream will not report the end of the split just yet. Once the
* next record is read then the next block will be accessed and the
* stream will indicate the end of the split. The consumer of the next
* split will correctly discard the first record of its split, and no
* data is lost or duplicated.
*
* If the default delimiter is used and the block ends at a CR then this
* is treated as case 2 since the reader does not yet know without
* looking at subsequent bytes whether the delimiter has ended.
*
* NOTE: It is assumed that compressed input streams *never* return bytes from
* multiple compressed blocks from a single read. Failure to do so will
* violate the buffering performed by this class, as it will access
* bytes into the next block after the split before returning all of the
* records from the previous block.
*/
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class CompressedSplitLineReader extends SplitLineReader {
SplitCompressionInputStream scin;
private boolean usingCRLF;
private boolean needAdditionalRecord = false;
private boolean finished = false;
public CompressedSplitLineReader(SplitCompressionInputStream in,
Configuration conf,
byte[] recordDelimiterBytes)
throws IOException {
super(in, conf, recordDelimiterBytes);
scin = in;
usingCRLF = (recordDelimiterBytes == null);
}
@Override
protected int fillBuffer(InputStream in, byte[] buffer, boolean inDelimiter)
throws IOException {
int bytesRead = in.read(buffer);
// If the split ended in the middle of a record delimiter then we need
// to read one additional record, as the consumer of the next split will
// not recognize the partial delimiter as a record.
// However if using the default delimiter and the next character is a
// linefeed then next split will treat it as a delimiter all by itself
// and the additional record read should not be performed.
if (inDelimiter && bytesRead > 0) {
if (usingCRLF) {
needAdditionalRecord = (buffer[0] != '\n');
} else {
needAdditionalRecord = true;
}
}
return bytesRead;
}
@Override
public int readLine(Text str, int maxLineLength, int maxBytesToConsume)
throws IOException {
int bytesRead = 0;
if (!finished) {
// only allow at most one more record to be read after the stream
// reports the split ended
if (scin.getPos() > scin.getAdjustedEnd()) {
finished = true;
}
bytesRead = super.readLine(str, maxLineLength, maxBytesToConsume);
}
return bytesRead;
}
@Override
public boolean needAdditionalRecordAfterSplit() {
return !finished && needAdditionalRecord;
}
}

View File

@ -38,7 +38,6 @@ import org.apache.hadoop.io.compress.Decompressor;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.util.LineReader;
import org.apache.commons.logging.LogFactory;
import org.apache.commons.logging.Log;
@ -55,7 +54,7 @@ public class LineRecordReader extends RecordReader<LongWritable, Text> {
private long start;
private long pos;
private long end;
private LineReader in;
private SplitLineReader in;
private FSDataInputStream fileIn;
private Seekable filePosition;
private int maxLineLength;
@ -94,33 +93,19 @@ public class LineRecordReader extends RecordReader<LongWritable, Text> {
((SplittableCompressionCodec)codec).createInputStream(
fileIn, decompressor, start, end,
SplittableCompressionCodec.READ_MODE.BYBLOCK);
if (null == this.recordDelimiterBytes){
in = new LineReader(cIn, job);
} else {
in = new LineReader(cIn, job, this.recordDelimiterBytes);
}
in = new CompressedSplitLineReader(cIn, job,
this.recordDelimiterBytes);
start = cIn.getAdjustedStart();
end = cIn.getAdjustedEnd();
filePosition = cIn;
} else {
if (null == this.recordDelimiterBytes) {
in = new LineReader(codec.createInputStream(fileIn, decompressor),
job);
} else {
in = new LineReader(codec.createInputStream(fileIn,
decompressor), job, this.recordDelimiterBytes);
}
in = new SplitLineReader(codec.createInputStream(fileIn,
decompressor), job, this.recordDelimiterBytes);
filePosition = fileIn;
}
} else {
fileIn.seek(start);
if (null == this.recordDelimiterBytes){
in = new LineReader(fileIn, job);
} else {
in = new LineReader(fileIn, job, this.recordDelimiterBytes);
}
in = new SplitLineReader(fileIn, job, this.recordDelimiterBytes);
filePosition = fileIn;
}
// If this is not the first split, we always throw away first record
@ -160,7 +145,7 @@ public class LineRecordReader extends RecordReader<LongWritable, Text> {
int newSize = 0;
// We always read one extra line, which lies outside the upper
// split limit i.e. (end - 1)
while (getFilePosition() <= end) {
while (getFilePosition() <= end || in.needAdditionalRecordAfterSplit()) {
newSize = in.readLine(value, maxLineLength,
Math.max(maxBytesToConsume(pos), maxLineLength));
pos += newSize;

View File

@ -0,0 +1,43 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.lib.input;
import java.io.IOException;
import java.io.InputStream;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
@InterfaceAudience.Private
@InterfaceStability.Unstable
public class SplitLineReader extends org.apache.hadoop.util.LineReader {
public SplitLineReader(InputStream in, byte[] recordDelimiterBytes) {
super(in, recordDelimiterBytes);
}
public SplitLineReader(InputStream in, Configuration conf,
byte[] recordDelimiterBytes) throws IOException {
super(in, conf, recordDelimiterBytes);
}
public boolean needAdditionalRecordAfterSplit() {
return false;
}
}

View File

@ -0,0 +1,100 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapred;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.io.File;
import java.io.IOException;
import java.net.URL;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text;
import org.junit.Test;
public class TestLineRecordReader {
private void testSplitRecords(String testFileName, long firstSplitLength)
throws IOException {
URL testFileUrl = getClass().getClassLoader().getResource(testFileName);
assertNotNull("Cannot find " + testFileName, testFileUrl);
File testFile = new File(testFileUrl.getFile());
long testFileSize = testFile.length();
Path testFilePath = new Path(testFile.getAbsolutePath());
Configuration conf = new Configuration();
conf.setInt(org.apache.hadoop.mapreduce.lib.input.
LineRecordReader.MAX_LINE_LENGTH, Integer.MAX_VALUE);
assertTrue("unexpected test data at " + testFile,
testFileSize > firstSplitLength);
// read the data without splitting to count the records
FileSplit split = new FileSplit(testFilePath, 0, testFileSize,
(String[])null);
LineRecordReader reader = new LineRecordReader(conf, split);
LongWritable key = new LongWritable();
Text value = new Text();
int numRecordsNoSplits = 0;
while (reader.next(key, value)) {
++numRecordsNoSplits;
}
reader.close();
// count the records in the first split
split = new FileSplit(testFilePath, 0, firstSplitLength, (String[])null);
reader = new LineRecordReader(conf, split);
int numRecordsFirstSplit = 0;
while (reader.next(key, value)) {
++numRecordsFirstSplit;
}
reader.close();
// count the records in the second split
split = new FileSplit(testFilePath, firstSplitLength,
testFileSize - firstSplitLength, (String[])null);
reader = new LineRecordReader(conf, split);
int numRecordsRemainingSplits = 0;
while (reader.next(key, value)) {
++numRecordsRemainingSplits;
}
reader.close();
assertEquals("Unexpected number of records in bzip2 compressed split",
numRecordsNoSplits, numRecordsFirstSplit + numRecordsRemainingSplits);
}
@Test
public void testBzip2SplitEndsAtCR() throws IOException {
// the test data contains a carriage-return at the end of the first
// split which ends at compressed offset 136498 and the next
// character is not a linefeed
testSplitRecords("blockEndingInCR.txt.bz2", 136498);
}
@Test
public void testBzip2SplitEndsAtCRThenLF() throws IOException {
// the test data contains a carriage-return at the end of the first
// split which ends at compressed offset 136498 and the next
// character is a linefeed
testSplitRecords("blockEndingInCRThenLF.txt.bz2", 136498);
}
}

View File

@ -0,0 +1,104 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.mapreduce.lib.input;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import java.io.File;
import java.io.IOException;
import java.net.URL;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.TaskAttemptID;
import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
import org.junit.Test;
public class TestLineRecordReader {
private void testSplitRecords(String testFileName, long firstSplitLength)
throws IOException {
URL testFileUrl = getClass().getClassLoader().getResource(testFileName);
assertNotNull("Cannot find " + testFileName, testFileUrl);
File testFile = new File(testFileUrl.getFile());
long testFileSize = testFile.length();
Path testFilePath = new Path(testFile.getAbsolutePath());
Configuration conf = new Configuration();
conf.setInt(org.apache.hadoop.mapreduce.lib.input.
LineRecordReader.MAX_LINE_LENGTH, Integer.MAX_VALUE);
assertTrue("unexpected test data at " + testFile,
testFileSize > firstSplitLength);
TaskAttemptContext context = new TaskAttemptContextImpl(conf, new TaskAttemptID());
// read the data without splitting to count the records
FileSplit split = new FileSplit(testFilePath, 0, testFileSize,
(String[])null);
LineRecordReader reader = new LineRecordReader();
reader.initialize(split, context);
int numRecordsNoSplits = 0;
while (reader.nextKeyValue()) {
++numRecordsNoSplits;
}
reader.close();
// count the records in the first split
split = new FileSplit(testFilePath, 0, firstSplitLength, (String[])null);
reader = new LineRecordReader();
reader.initialize(split, context);
int numRecordsFirstSplit = 0;
while (reader.nextKeyValue()) {
++numRecordsFirstSplit;
}
reader.close();
// count the records in the second split
split = new FileSplit(testFilePath, firstSplitLength,
testFileSize - firstSplitLength, (String[])null);
reader = new LineRecordReader();
reader.initialize(split, context);
int numRecordsRemainingSplits = 0;
while (reader.nextKeyValue()) {
++numRecordsRemainingSplits;
}
reader.close();
assertEquals("Unexpected number of records in bzip2 compressed split",
numRecordsNoSplits, numRecordsFirstSplit + numRecordsRemainingSplits);
}
@Test
public void testBzip2SplitEndsAtCR() throws IOException {
// the test data contains a carriage-return at the end of the first
// split which ends at compressed offset 136498 and the next
// character is not a linefeed
testSplitRecords("blockEndingInCR.txt.bz2", 136498);
}
@Test
public void testBzip2SplitEndsAtCRThenLF() throws IOException {
// the test data contains a carriage-return at the end of the first
// split which ends at compressed offset 136498 and the next
// character is a linefeed
testSplitRecords("blockEndingInCRThenLF.txt.bz2", 136498);
}
}

View File

@ -53,7 +53,8 @@ public class PartialJob implements org.apache.hadoop.mapreduce.v2.app.job.Job {
this.jobIndexInfo = jobIndexInfo;
this.jobId = jobId;
jobReport = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(JobReport.class);
jobReport.setStartTime(jobIndexInfo.getSubmitTime());
jobReport.setSubmitTime(jobIndexInfo.getSubmitTime());
jobReport.setStartTime(jobIndexInfo.getJobStartTime());
jobReport.setFinishTime(jobIndexInfo.getFinishTime());
jobReport.setJobState(getState());
}

View File

@ -84,6 +84,7 @@ public class HsJobBlock extends HtmlBlock {
_("Queue:", job.getQueueName()).
_("State:", job.getState()).
_("Uberized:", job.isUber()).
_("Submitted:", new Date(job.getSubmitTime())).
_("Started:", new Date(job.getStartTime())).
_("Finished:", new Date(job.getFinishTime())).
_("Elapsed:", StringUtils.formatTime(

View File

@ -55,6 +55,7 @@ public class HsJobsBlock extends HtmlBlock {
table("#jobs").
thead().
tr().
th("Submit Time").
th("Start Time").
th("Finish Time").
th(".id", "Job ID").
@ -74,6 +75,7 @@ public class HsJobsBlock extends HtmlBlock {
for (Job j : appContext.getAllJobs().values()) {
JobInfo job = new JobInfo(j);
jobsTableData.append("[\"")
.append(dateFormat.format(new Date(job.getSubmitTime()))).append("\",\"")
.append(dateFormat.format(new Date(job.getStartTime()))).append("\",\"")
.append(dateFormat.format(new Date(job.getFinishTime()))).append("\",\"")
.append("<a href='").append(url("job", job.getId())).append("'>")
@ -101,6 +103,7 @@ public class HsJobsBlock extends HtmlBlock {
tbody._().
tfoot().
tr().
th().input("search_init").$type(InputType.text).$name("submit_time").$value("Submit Time")._()._().
th().input("search_init").$type(InputType.text).$name("start_time").$value("Start Time")._()._().
th().input("search_init").$type(InputType.text).$name("finish_time").$value("Finish Time")._()._().
th().input("search_init").$type(InputType.text).$name("start_time").$value("Job ID")._()._().

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.security.authorize.AccessControlList;
@XmlAccessorType(XmlAccessType.FIELD)
public class JobInfo {
protected long submitTime;
protected long startTime;
protected long finishTime;
protected String id;
@ -85,6 +86,7 @@ public class JobInfo {
this.mapsCompleted = job.getCompletedMaps();
this.reducesTotal = job.getTotalReduces();
this.reducesCompleted = job.getCompletedReduces();
this.submitTime = report.getSubmitTime();
this.startTime = report.getStartTime();
this.finishTime = report.getFinishTime();
this.name = job.getName().toString();
@ -216,6 +218,10 @@ public class JobInfo {
return this.id;
}
public long getSubmitTime() {
return this.submitTime;
}
public long getStartTime() {
return this.startTime;
}

View File

@ -86,6 +86,7 @@ public class MockHistoryJobs extends MockJobs {
report.getFinishTime(), mockJob.getUserName(), mockJob.getName(), id,
mockJob.getCompletedMaps(), mockJob.getCompletedReduces(),
String.valueOf(mockJob.getState()));
info.setJobStartTime(report.getStartTime());
info.setQueueName(mockJob.getQueueName());
ret.partial.put(id, new PartialJob(info, id));

View File

@ -33,7 +33,7 @@ import org.codehaus.jettison.json.JSONObject;
public class VerifyJobsUtils {
public static void verifyHsJobPartial(JSONObject info, Job job) throws JSONException {
assertEquals("incorrect number of elements", 11, info.length());
assertEquals("incorrect number of elements", 12, info.length());
// everyone access fields
verifyHsJobGeneric(job, info.getString("id"), info.getString("user"),
@ -45,7 +45,7 @@ public class VerifyJobsUtils {
}
public static void verifyHsJob(JSONObject info, Job job) throws JSONException {
assertEquals("incorrect number of elements", 24, info.length());
assertEquals("incorrect number of elements", 25, info.length());
// everyone access fields
verifyHsJobGeneric(job, info.getString("id"), info.getString("user"),

View File

@ -201,6 +201,7 @@ History Server REST API's.
"jobs" : {
"job" : [
{
"submitTime" : 1326381344449,
"state" : "SUCCEEDED",
"user" : "user1",
"reducesTotal" : 1,
@ -214,6 +215,7 @@ History Server REST API's.
"finishTime" : 1326381356010
},
{
"submitTime" : 1326381446500
"state" : "SUCCEEDED",
"user" : "user1",
"reducesTotal" : 1,
@ -255,6 +257,7 @@ History Server REST API's.
<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
<jobs>
<job>
<submitTime>1326381344449</submitTime>
<startTime>1326381344489</startTime>
<finishTime>1326381356010</finishTime>
<id>job_1326381300833_1_1</id>
@ -268,6 +271,7 @@ History Server REST API's.
<reducesCompleted>1</reducesCompleted>
</job>
<job>
<submitTime>1326381446500</submitTime>
<startTime>1326381446529</startTime>
<finishTime>1326381582106</finishTime>
<id>job_1326381300833_2_2</id>
@ -322,6 +326,8 @@ History Server REST API's.
*---------------+--------------+-------------------------------+
| diagnostics | string | A diagnostic message |
*---------------+--------------+-------------------------------+
| submitTime | long | The time the job submitted (in ms since epoch)|
*---------------+--------------+-------------------------------+
| startTime | long | The time the job started (in ms since epoch)|
*---------------+--------------+-------------------------------+
| finishTime | long | The time the job finished (in ms since epoch)|
@ -393,6 +399,7 @@ History Server REST API's.
+---+
{
"job" : {
"submitTime": 1326381446500,
"avgReduceTime" : 124961,
"failedReduceAttempts" : 0,
"state" : "SUCCEEDED",
@ -453,6 +460,7 @@ History Server REST API's.
+---+
<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
<job>
<submitTime>1326381446500</submitTime>
<startTime>1326381446529</startTime>
<finishTime>1326381582106</finishTime>
<id>job_1326381300833_2_2</id>
@ -2663,5 +2671,4 @@ History Server REST API's.
</counter>
</taskAttemptCounterGroup>
</jobTaskAttemptCounters>
+---+
+---+