cleanup client error message propagation
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ExceptionFilterUtils.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ExceptionFilterUtils.java
new file mode 100644
index 0000000..44f3bec
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ExceptionFilterUtils.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.hyracks.control.cc.job;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * @author yingyib
+ */
+public class ExceptionFilterUtils {
+
+    public static List<Exception> getActualExceptions(List<Exception> allExceptions) {
+        List<Exception> exceptions = new ArrayList<Exception>();
+        for (Exception exception : allExceptions) {
+            if (possibleRootCause(exception)) {
+                exceptions.add(exception);
+            }
+        }
+        return exceptions;
+    }
+
+    private static boolean possibleRootCause(Throwable exception) {
+        Throwable cause = exception;
+        while ((cause = cause.getCause()) != null) {
+            if (cause instanceof java.lang.InterruptedException
+                    || cause instanceof java.nio.channels.ClosedChannelException) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+}
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
index 9cd7886..2166620 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobRun.java
@@ -347,7 +347,9 @@
                                 taskAttempt.put("end-time", ta.getEndTime());
                                 List<Exception> exceptions = ta.getExceptions();
                                 if (exceptions != null && !exceptions.isEmpty()) {
-                                    for(Exception exception : exceptions){
+                                    List<Exception> filteredExceptions = ExceptionFilterUtils
+                                            .getActualExceptions(exceptions);
+                                    for (Exception exception : filteredExceptions) {
                                         StringWriter exceptionWriter = new StringWriter();
                                         exception.printStackTrace(new PrintWriter(exceptionWriter));
                                         taskAttempt.put("failure-details", exceptionWriter.toString());
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
index 3b29b52..6e8ddf0 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
@@ -59,7 +59,9 @@
         }
         Set<String> targetNodes = run.getParticipatingNodeIds();
         run.getCleanupPendingNodeIds().addAll(targetNodes);
-        run.setPendingStatus(status, exceptions);
+        if (run.getPendingStatus() != JobStatus.FAILURE && run.getPendingStatus() != JobStatus.TERMINATED) {
+            run.setPendingStatus(status, exceptions);
+        }
         if (targetNodes != null && !targetNodes.isEmpty()) {
             for (String n : targetNodes) {
                 NodeControllerState ncs = ccs.getNodeMap().get(n);
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java
index 07d2d57..a280c45 100644
--- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/ConnectedComponentsVertex.java
@@ -139,6 +139,7 @@
         job.setVertexOutputFormatClass(SimpleConnectedComponentsVertexOutputFormat.class);
         job.setMessageCombinerClass(ConnectedComponentsVertex.SimpleMinCombiner.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+        job.setDynamicVertexValueSize(true);
         Client.run(args, job);
     }
 
diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/FailureVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/FailureVertex.java
new file mode 100644
index 0000000..d2464c1
--- /dev/null
+++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/FailureVertex.java
@@ -0,0 +1,36 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.pregelix.example;
+
+import java.util.Iterator;
+
+import org.apache.hadoop.io.FloatWritable;
+
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.example.io.VLongWritable;
+
+/**
+ * @author yingyib
+ */
+public class FailureVertex extends Vertex<VLongWritable, VLongWritable, FloatWritable, VLongWritable> {
+
+    @Override
+    public void compute(Iterator<VLongWritable> msgIterator) throws Exception {
+        if (getVertexId().get() == 10) {
+            throw new IllegalStateException("This job is going to fail");
+        }
+    }
+
+}
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
index 13cec61..cb7fd6d 100644
--- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java
@@ -126,6 +126,7 @@
         job.setVertexOutputFormatClass(SimpleConnectedComponentsVertexOutputFormat.class);
         job.setMessageCombinerClass(ConnectedComponentsVertex.SimpleMinCombiner.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+        job.setDynamicVertexValueSize(true);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
@@ -140,6 +141,7 @@
         job.setMessageCombinerClass(ConnectedComponentsVertex.SimpleMinCombiner.class);
         job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
         job.setVertexPartitionerClass(DefaultVertexPartitioner.class);
+        job.setDynamicVertexValueSize(true);
         FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2);
         FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2));
         job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/test/FailureVertexTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/test/FailureVertexTest.java
new file mode 100644
index 0000000..f5116ca
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/test/FailureVertexTest.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.pregelix.example.test;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.junit.Test;
+
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.core.driver.Driver;
+import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
+import edu.uci.ics.pregelix.example.ConnectedComponentsVertex;
+import edu.uci.ics.pregelix.example.ConnectedComponentsVertex.SimpleConnectedComponentsVertexOutputFormat;
+import edu.uci.ics.pregelix.example.FailureVertex;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
+import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
+
+/**
+ * @author yingyib
+ */
+public class FailureVertexTest {
+
+    private static String HDFS_INPUTPATH2 = "data/webmapcomplex";
+    private static String HDFS_OUTPUTPAH2 = "actual/resultcomplex";
+
+    @Test
+    public void test() throws Exception {
+        TestCluster testCluster = new TestCluster();
+        try {
+            PregelixJob job = new PregelixJob(FailureVertex.class.getSimpleName());
+            job.setVertexClass(FailureVertex.class);
+            job.setVertexInputFormatClass(TextPageRankInputFormat.class);
+            job.setVertexOutputFormatClass(SimpleConnectedComponentsVertexOutputFormat.class);
+            job.setMessageCombinerClass(ConnectedComponentsVertex.SimpleMinCombiner.class);
+            job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+            job.setDynamicVertexValueSize(true);
+
+            FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2);
+            FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2));
+            job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
+
+            Driver driver = new Driver(FailureVertex.class);
+            testCluster.setUp();
+            driver.runJob(job, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT);
+        } catch (Exception e) {
+            Assert.assertTrue(e.toString().contains("This job is going to fail"));
+        } finally {
+            testCluster.tearDown();
+        }
+    }
+
+}
diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/test/TestCluster.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/test/TestCluster.java
new file mode 100644
index 0000000..d0cf654
--- /dev/null
+++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/test/TestCluster.java
@@ -0,0 +1,150 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed 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 from
+ * 
+ *     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 edu.uci.ics.pregelix.example.test;
+
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.logging.Logger;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
+import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
+
+@SuppressWarnings("deprecation")
+public class TestCluster {
+    private static final Logger LOGGER = Logger.getLogger(TestCluster.class.getName());
+
+    private static final String ACTUAL_RESULT_DIR = "actual";
+    private static final String PATH_TO_HADOOP_CONF = "src/test/resources/hadoop/conf";
+    private static final String PATH_TO_CLUSTER_STORE = "src/test/resources/cluster/stores.properties";
+    private static final String PATH_TO_CLUSTER_PROPERTIES = "src/test/resources/cluster/cluster.properties";
+
+    private static final String DATA_PATH = "data/webmap/webmap_link.txt";
+    private static final String HDFS_PATH = "/webmap/";
+
+    private static final String DATA_PATH2 = "data/webmapcomplex/webmap_link.txt";
+    private static final String HDFS_PATH2 = "/webmapcomplex/";
+
+    private static final String DATA_PATH3 = "data/clique/clique.txt";
+    private static final String HDFS_PATH3 = "/clique/";
+
+    private static final String DATA_PATH4 = "data/clique2/clique.txt";
+    private static final String HDFS_PATH4 = "/clique2/";
+
+    private static final String DATA_PATH5 = "data/clique3/clique.txt";
+    private static final String HDFS_PATH5 = "/clique3/";
+
+    private static final String HADOOP_CONF_PATH = ACTUAL_RESULT_DIR + File.separator + "conf.xml";
+    private MiniDFSCluster dfsCluster;
+
+    private JobConf conf = new JobConf();
+    private int numberOfNC = 2;
+
+    public void setUp() throws Exception {
+        ClusterConfig.setStorePath(PATH_TO_CLUSTER_STORE);
+        ClusterConfig.setClusterPropertiesPath(PATH_TO_CLUSTER_PROPERTIES);
+        cleanupStores();
+        PregelixHyracksIntegrationUtil.init();
+        LOGGER.info("Hyracks mini-cluster started");
+        FileUtils.forceMkdir(new File(ACTUAL_RESULT_DIR));
+        FileUtils.cleanDirectory(new File(ACTUAL_RESULT_DIR));
+        startHDFS();
+    }
+
+    private void cleanupStores() throws IOException {
+        FileUtils.forceMkdir(new File("teststore"));
+        FileUtils.forceMkdir(new File("build"));
+        FileUtils.cleanDirectory(new File("teststore"));
+        FileUtils.cleanDirectory(new File("build"));
+    }
+
+    private void startHDFS() throws IOException {
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/core-site.xml"));
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/mapred-site.xml"));
+        conf.addResource(new Path(PATH_TO_HADOOP_CONF + "/hdfs-site.xml"));
+        FileSystem lfs = FileSystem.getLocal(new Configuration());
+        lfs.delete(new Path("build"), true);
+        System.setProperty("hadoop.log.dir", "logs");
+        dfsCluster = new MiniDFSCluster(conf, numberOfNC, true, null);
+        FileSystem dfs = FileSystem.get(conf);
+        Path src = new Path(DATA_PATH);
+        Path dest = new Path(HDFS_PATH);
+        dfs.mkdirs(dest);
+        dfs.copyFromLocalFile(src, dest);
+
+        src = new Path(DATA_PATH2);
+        dest = new Path(HDFS_PATH2);
+        dfs.mkdirs(dest);
+        dfs.copyFromLocalFile(src, dest);
+
+        src = new Path(DATA_PATH3);
+        dest = new Path(HDFS_PATH3);
+        dfs.mkdirs(dest);
+        dfs.copyFromLocalFile(src, dest);
+
+        src = new Path(DATA_PATH4);
+        dest = new Path(HDFS_PATH4);
+        dfs.mkdirs(dest);
+        dfs.copyFromLocalFile(src, dest);
+
+        src = new Path(DATA_PATH5);
+        dest = new Path(HDFS_PATH5);
+        dfs.mkdirs(dest);
+        dfs.copyFromLocalFile(src, dest);
+
+        DataOutputStream confOutput = new DataOutputStream(new FileOutputStream(new File(HADOOP_CONF_PATH)));
+        conf.writeXml(confOutput);
+        confOutput.flush();
+        confOutput.close();
+    }
+
+    /**
+     * cleanup hdfs cluster
+     */
+    private void cleanupHDFS() throws Exception {
+        dfsCluster.shutdown();
+    }
+
+    public void tearDown() throws Exception {
+        PregelixHyracksIntegrationUtil.deinit();
+        LOGGER.info("Hyracks mini-cluster shut down");
+        cleanupHDFS();
+    }
+
+    protected static List<String> getFileList(String ignorePath) throws FileNotFoundException, IOException {
+        BufferedReader reader = new BufferedReader(new FileReader(ignorePath));
+        String s = null;
+        List<String> ignores = new ArrayList<String>();
+        while ((s = reader.readLine()) != null) {
+            ignores.add(s);
+        }
+        reader.close();
+        return ignores;
+    }
+
+}
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml
index decbde8..df72d9b 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml
@@ -127,6 +127,7 @@
 <property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.inputformat.TextConnectedComponentsInputFormat</value></property>
 <property><name>mapred.job.queue.name</name><value>default</value></property>
 <property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
+<property><name>pregelix.incStateLength</name><value>true</value></property>
 <property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
 <property><name>topology.script.number.args</name><value>100</value></property>
 <property><name>mapred.skip.map.max.skip.records</name><value>0</value></property>
diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml
index cca66bb..b0bf024 100644
--- a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml
+++ b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml
@@ -128,6 +128,7 @@
 <property><name>pregelix.vertexInputFormatClass</name><value>edu.uci.ics.pregelix.example.inputformat.TextConnectedComponentsInputFormat</value></property>
 <property><name>mapred.job.queue.name</name><value>default</value></property>
 <property><name>mapred.job.tracker.persist.jobstatus.active</name><value>false</value></property>
+<property><name>pregelix.incStateLength</name><value>true</value></property>
 <property><name>mapred.reduce.slowstart.completed.maps</name><value>0.05</value></property>
 <property><name>topology.script.number.args</name><value>100</value></property>
 <property><name>mapred.skip.map.max.skip.records</name><value>0</value></property>