Skip to content

Commit

Permalink
Merge pull request #85 from coderplay/minzhou
Browse files Browse the repository at this point in the history
Merge code from Min zhou, includes #73 #82  #80 #79
  • Loading branch information
coderplay committed Jul 25, 2012
2 parents 334bd3e + a113c9d commit b7af2c4
Show file tree
Hide file tree
Showing 34 changed files with 1,225 additions and 1,154 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,8 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.realtime.conf.DragonConfiguration;
import org.apache.hadoop.realtime.mr.Mapper;
import org.apache.hadoop.realtime.mr.Reducer;
import org.apache.hadoop.realtime.records.CounterGroup;
import org.apache.hadoop.realtime.records.Counters;
import org.apache.hadoop.realtime.records.JobId;
Expand Down Expand Up @@ -322,11 +324,25 @@ public JobState getState() {
return null;
}

public void setMapper(Class<?> clazz){
conf.setClass(DragonJobConfig.JOB_MAP_CLASS, clazz, Object.class);
/**
* Set the {@link Mapper} for the job.
* @param cls the <code>Mapper</code> to use
* @throws IllegalStateException if the job is submitted
*/
public void setMapperClass(Class<? extends Mapper> clazz)
throws IllegalStateException {
ensureState(JobState.NEW);
conf.setClass(DragonJobConfig.JOB_MAP_CLASS, clazz, Mapper.class);
}

public void setReducer(Class<?> clazz){
conf.setClass(DragonJobConfig.JOB_REDUCE_CLASS, clazz, Object.class);

/**
* Set the {@link Reducer} for the job.
* @param cls the <code>Reducer</code> to use
* @throws IllegalStateException if the job is submitted
*/
public void setReducerClass(Class<? extends Reducer> clazz)
throws IllegalStateException {
ensureState(JobState.NEW);
conf.setClass(DragonJobConfig.JOB_REDUCE_CLASS, clazz, Reducer.class);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -292,11 +292,13 @@ public class DragonJobConfig {
public static final int DEFAULT_JOB_HISTORY_USE_BATCHED_FLUSH_QUEUE_SIZE_THRESHOLD =
50;

public static final String JOB_MAP_CLASS = "map.class";
public static final String JOB_MAP_CLASS = "dragon.map.class";

public static final String JOB_REDUCE_CLASS = "reduce.class";
public static final String JOB_REDUCE_CLASS = "dragon.reduce.class";

public static final String MAP_PARALLELISM = "map.parallelism";
public static final String JOB_OUTPUT_DIR = "dragon.output.dir";

public static final String MAP_PARALLELISM = "dragon.map.tasks";

public static final String REDUCE_PARALLELISM = "reduce.parallelism";
public static final String REDUCE_PARALLELISM = "dragon.reduce.tasks";
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,17 +17,219 @@
*/
package org.apache.hadoop.realtime.child;

import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.TimeUnit;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileSystem.Statistics;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.realtime.DragonJobConfig;
import org.apache.hadoop.realtime.conf.DragonConfiguration;
import org.apache.hadoop.realtime.event.Event;
import org.apache.hadoop.realtime.event.EventEmitter;
import org.apache.hadoop.realtime.event.EventProducer;
import org.apache.hadoop.realtime.mr.MapContext;
import org.apache.hadoop.realtime.mr.Mapper;
import org.apache.hadoop.realtime.protocol.DragonChildProtocol;
import org.apache.hadoop.realtime.records.ChildExecutionContext;
import org.apache.hadoop.realtime.records.Counters;
import org.apache.hadoop.realtime.records.TaskType;
import org.apache.hadoop.util.ReflectionUtils;

/**
*/
public class ChildExecutor {
abstract class ChildExecutor {
// @InterfaceAudience.Private
// @InterfaceStability.Unstable
// protected class TaskReporter implements StatusReporter, Runnable {
// private DragonChildProtocol umbilical;;
// private DragonConfiguration conf;
// private Progress taskProgress;
// private Thread pingThread = null;
// private boolean done = true;
// private Object lock = new Object();
//
// TaskReporter(Progress taskProgress, DragonChildProtocol umbilical) {
// this.umbilical = umbilical;
// this.taskProgress = taskProgress;
// }
//
//
// public void setStatus(String status) {
// taskProgress.setStatus(normalizeStatus(status, conf));
// }
//
// public Counters.Counter getCounter(String group, String name) {
// Counters.Counter counter = null;
// if (counters != null) {
// counter = counters.findCounter(group, name);
// }
// return counter;
// }
//
// public Counters.Counter getCounter(Enum<?> name) {
// return counters == null ? null : counters.findCounter(name);
// }
//
// public void incrCounter(Enum key, long amount) {
// if (counters != null) {
// counters.incrCounter(key, amount);
// }
// }
//
// public void incrCounter(String group, String counter, long amount) {
// if (counters != null) {
// counters.incrCounter(group, counter, amount);
// }
// if (skipping
// && SkipBadRecords.COUNTER_GROUP.equals(group)
// && (SkipBadRecords.COUNTER_MAP_PROCESSED_RECORDS.equals(counter) ||
// SkipBadRecords.COUNTER_REDUCE_PROCESSED_GROUPS
// .equals(counter))) {
// // if application reports the processed records, move the
// // currentRecStartIndex to the next.
// // currentRecStartIndex is the start index which has not yet been
// // finished and is still in task's stomach.
// for (int i = 0; i < amount; i++) {
// currentRecStartIndex = currentRecIndexIterator.next();
// }
// }
// setProgressFlag();
// }
//
// /**
// * The communication thread handles communication with the parent (Task
// * Tracker). It sends progress updates if progress has been made or if the
// * task needs to let the parent know that it's alive. It also pings the
// * parent to see if it's alive.
// */
// public void run() {
// final int MAX_RETRIES = 3;
// int remainingRetries = MAX_RETRIES;
// // get current flag value and reset it as well
// boolean sendProgress = resetProgressFlag();
// while (!taskDone.get()) {
// synchronized (lock) {
// done = false;
// }
// try {
// boolean taskFound = true; // whether TT knows about this task
// // sleep for a bit
// synchronized (lock) {
// if (taskDone.get()) {
// break;
// }
// lock.wait(PROGRESS_INTERVAL);
// }
// if (taskDone.get()) {
// break;
// }
//
// if (sendProgress) {
// // we need to send progress update
// updateCounters();
// taskStatus.statusUpdate(taskProgress.get(),
// taskProgress.toString(), counters);
// taskFound = umbilical.statusUpdate(taskId, taskStatus);
// taskStatus.clearStatus();
// } else {
// // send ping
// taskFound = umbilical.ping(taskId);
// }
//
// // if Task Tracker is not aware of our task ID (probably because it
// // died and
// // came back up), kill ourselves
// if (!taskFound) {
// LOG.warn("Parent died. Exiting " + taskId);
// resetDoneFlag();
// System.exit(66);
// }
//
// sendProgress = resetProgressFlag();
// remainingRetries = MAX_RETRIES;
// } catch (Throwable t) {
// LOG.info("Communication exception: "
// + StringUtils.stringifyException(t));
// remainingRetries -= 1;
// if (remainingRetries == 0) {
// ReflectionUtils.logThreadInfo(LOG, "Communication exception", 0);
// LOG.warn("Last retry, killing " + taskId);
// resetDoneFlag();
// System.exit(65);
// }
// }
// }
// // Notify that we are done with the work
// resetDoneFlag();
// }
//
// void resetDoneFlag() {
// synchronized (lock) {
// done = true;
// lock.notify();
// }
// }
//
// public void startCommunicationThread() {
// if (pingThread == null) {
// pingThread = new Thread(this, "communication thread");
// pingThread.setDaemon(true);
// pingThread.start();
// }
// }
//
// public void stopCommunicationThread() throws InterruptedException {
// if (pingThread != null) {
// // Intent of the lock is to not send an interupt in the middle of an
// // umbilical.ping or umbilical.statusUpdate
// synchronized (lock) {
// // Interrupt if sleeping. Otherwise wait for the RPC call to return.
// lock.notify();
// }
//
// synchronized (lock) {
// while (!done) {
// lock.wait();
// }
// }
// pingThread.interrupt();
// pingThread.join();
// }
// }
// }

public static void run(Configuration conf, DragonChildProtocol proxy,
ChildExecutionContext context) {
/**
* Gets a handle to the Statistics instance based on the scheme associated
* with path.
*
* @param path the path.
* @param conf the configuration to extract the scheme from if not part of the
* path.
* @return a Statistics instance, or null if none is found for the scheme.
*/
protected static List<Statistics> getFsStatistics(Path path,
Configuration conf) throws IOException {
List<Statistics> matchedStats = new ArrayList<FileSystem.Statistics>();
path = path.getFileSystem(conf).makeQualified(path);
String scheme = path.toUri().getScheme();
for (Statistics stats : FileSystem.getAllStatistics()) {
if (stats.getScheme().equals(scheme)) {
matchedStats.add(stats);
}
}
return matchedStats;
}


protected <KEYIN, VALUEIN, KEYOUT, VALUEOUT> void execute(
final Configuration conf, final DragonChildProtocol proxy,
final ChildExecutionContext context) throws IOException,
InterruptedException {
// make a config helper so we can get the classes
}


}
Original file line number Diff line number Diff line change
Expand Up @@ -15,11 +15,19 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.realtime.event;
package org.apache.hadoop.realtime.child;

import org.apache.hadoop.realtime.records.TaskType;

/**
* Receives and persists the terminal {@link Event}
*/
public interface EventSource {
public void nextEvent();
public final class ChildExecutorFactory {
public static ChildExecutor newExecutor(TaskType type) {
if (type == TaskType.MAP)
return new MapChildExecutor();
else if (type == TaskType.MAP)
return new ReduceChildExecutor();
else
return null; // or throw an exception?
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,92 @@
/**
* 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.realtime.child;

import java.io.IOException;
import java.util.concurrent.TimeUnit;

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.realtime.conf.DragonConfiguration;
import org.apache.hadoop.realtime.event.Event;
import org.apache.hadoop.realtime.mr.Mapper;
import org.apache.hadoop.realtime.mr.Reducer;
import org.apache.hadoop.realtime.records.TaskAttemptId;
import org.apache.hadoop.realtime.records.TaskType;

/**
* A context object that allows input and output from the task. It is only
* supplied to the {@link Mapper} or {@link Reducer}.
* @param <KEYIN> the input key type for the task
* @param <VALUEIN> the input value type for the task
* @param <KEYOUT> the output key type for the task
* @param <VALUEOUT> the output value type for the task
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface Context<KEYIN,VALUEIN,KEYOUT,VALUEOUT> {

DragonConfiguration getConfiguration();

TaskAttemptId getTaskAttemptId();

/**
* return the partition number of this task
* @return
*/
int getPartition();

String getUser();


TaskType getTaskType();

/**
* take a event from the queue.
*
* @return the current key object or null if there isn't one
* @throws IOException
* @throws InterruptedException
*/
public Event<KEYIN, VALUEIN> pollEvent() throws IOException,
InterruptedException;

/**
* take a event from event producer
*
* @return the current key object or null if there isn't one
* @throws IOException
* @throws InterruptedException
*/
public Event<KEYIN, VALUEIN> pollEvent(long timeout, TimeUnit unit)
throws IOException, InterruptedException;

/**
* Generate an output event
*/
public boolean emitEvent(Event<KEYOUT, VALUEOUT> event) throws IOException,
InterruptedException;

/**
* Generate an output event
*/
public boolean emitEvent(Event<KEYOUT, VALUEOUT> event, long timeout,
TimeUnit unit) throws IOException, InterruptedException;

}
Loading

0 comments on commit b7af2c4

Please sign in to comment.