Skip to content

Commit

Permalink
setStatus and counter increment compatibility
Browse files Browse the repository at this point in the history
  • Loading branch information
plelevier committed Aug 26, 2013
1 parent c75c74d commit ffc0cdf
Show file tree
Hide file tree
Showing 2 changed files with 15 additions and 16 deletions.
Expand Up @@ -6,17 +6,18 @@
import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.Mapper;


import com.linkedin.camus.etl.kafka.common.EtlKey; import com.linkedin.camus.etl.kafka.common.EtlKey;
import com.twitter.elephantbird.util.HadoopCompat;


/** /**
* KafkaETL mapper * KafkaETL mapper
* *
* input -- EtlKey, AvroWrapper * input -- EtlKey, AvroWrapper
* *
* output -- EtlKey, AvroWrapper * output -- EtlKey, AvroWrapper
* *
*/ */
public class EtlMapper extends Mapper<EtlKey, AvroWrapper<Object>, EtlKey, AvroWrapper<Object>> { public class EtlMapper extends Mapper<EtlKey, AvroWrapper<Object>, EtlKey, AvroWrapper<Object>> {

@Override @Override
public void map(EtlKey key, AvroWrapper<Object> val, Context context) throws IOException, InterruptedException { public void map(EtlKey key, AvroWrapper<Object> val, Context context) throws IOException, InterruptedException {
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
Expand All @@ -25,6 +26,6 @@ public void map(EtlKey key, AvroWrapper<Object> val, Context context) throws IOE


long endTime = System.currentTimeMillis(); long endTime = System.currentTimeMillis();
long mapTime = ((endTime - startTime)); long mapTime = ((endTime - startTime));
context.getCounter("total", "mapper-time(ms)").increment(mapTime); HadoopCompat.incrementCounter(context.getCounter("total", "mapper-time(ms)"), mapTime);
} }
} }
Expand Up @@ -219,7 +219,7 @@ public boolean nextKeyValue() throws IOException, InterruptedException {
statusMsg += statusMsg.length() > 0 ? "; " : ""; statusMsg += statusMsg.length() > 0 ? "; " : "";
statusMsg += request.getTopic() + ":" + request.getNodeId() + ":" statusMsg += request.getTopic() + ":" + request.getNodeId() + ":"
+ request.getPartition(); + request.getPartition();
context.setStatus(statusMsg); HadoopCompat.setStatus(context, statusMsg);


if (reader != null) { if (reader != null) {
closeReader(); closeReader();
Expand All @@ -233,8 +233,8 @@ public boolean nextKeyValue() throws IOException, InterruptedException {


while (reader.getNext(key, msgValue)) { while (reader.getNext(key, msgValue)) {
context.progress(); context.progress();
mapperContext.getCounter("total", "data-read").increment(msgValue.getLength()); HadoopCompat.incrementCounter(mapperContext.getCounter("total", "data-read"), msgValue.getLength());
mapperContext.getCounter("total", "event-count").increment(1); HadoopCompat.incrementCounter(mapperContext.getCounter("total", "event-count"), 1);
byte[] bytes = getBytes(msgValue); byte[] bytes = getBytes(msgValue);


// check the checksum of message // check the checksum of message
Expand Down Expand Up @@ -280,32 +280,30 @@ public boolean nextKeyValue() throws IOException, InterruptedException {
} }


if (timeStamp < beginTimeStamp) { if (timeStamp < beginTimeStamp) {
mapperContext.getCounter("total", "skip-old").increment(1); HadoopCompat.incrementCounter(mapperContext.getCounter("total", "skip-old"), 1);
} else if (endTimeStamp == 0) { } else if (endTimeStamp == 0) {
DateTime time = new DateTime(timeStamp); DateTime time = new DateTime(timeStamp);
statusMsg += " begin read at " + time.toString(); statusMsg += " begin read at " + time.toString();
context.setStatus(statusMsg); HadoopCompat.setStatus(context, statusMsg);
System.out.println(key.getTopic() + " begin read at " + time.toString()); System.out.println(key.getTopic() + " begin read at " + time.toString());
endTimeStamp = (time.plusHours(this.maxPullHours)).getMillis(); endTimeStamp = (time.plusHours(this.maxPullHours)).getMillis();
} else if (timeStamp > endTimeStamp || System.currentTimeMillis() > maxPullTime) { } else if (timeStamp > endTimeStamp || System.currentTimeMillis() > maxPullTime) {
statusMsg += " max read at " + new DateTime(timeStamp).toString(); statusMsg += " max read at " + new DateTime(timeStamp).toString();
context.setStatus(statusMsg); HadoopCompat.setStatus(context, statusMsg);
System.out.println(key.getTopic() + " max read at " System.out.println(key.getTopic() + " max read at "
+ new DateTime(timeStamp).toString()); + new DateTime(timeStamp).toString());
mapperContext.getCounter("total", "request-time(ms)").increment( HadoopCompat.incrementCounter(mapperContext.getCounter("total", "request-time(ms)"), reader.getFetchTime());
reader.getFetchTime());
closeReader(); closeReader();
} }


long secondTime = System.currentTimeMillis(); long secondTime = System.currentTimeMillis();
value.datum(wrapper.getRecord()); value.datum(wrapper.getRecord());
long decodeTime = ((secondTime - tempTime)); long decodeTime = ((secondTime - tempTime));


mapperContext.getCounter("total", "decode-time(ms)").increment(decodeTime); HadoopCompat.incrementCounter(mapperContext.getCounter("total", "decode-time(ms)"), decodeTime);


if (reader != null) { if (reader != null) {
mapperContext.getCounter("total", "request-time(ms)").increment( HadoopCompat.incrementCounter(mapperContext.getCounter("total", "request-time(ms)"), reader.getFetchTime());
reader.getFetchTime());
} }
return true; return true;
} }
Expand Down

0 comments on commit ffc0cdf

Please sign in to comment.