merged master back
diff --git a/pom.xml b/pom.xml
index de5ef3b..d748d1f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -12,8 +12,7 @@
! 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.
- !-->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+ !--><project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>pregelix</artifactId>
@@ -22,7 +21,7 @@
<name>pregelix</name>
<properties>
- <jvm.extraargs />
+ <jvm.extraargs/>
</properties>
<profiles>
@@ -107,6 +106,7 @@
<module>pregelix-runtime</module>
<module>pregelix-core</module>
<module>pregelix-example</module>
+ <module>pregelix-benchmark</module>
<module>pregelix-dist</module>
</modules>
</project>
diff --git a/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java b/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
index 1e0d87a..6549c52 100644
--- a/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
+++ b/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java
@@ -76,6 +76,10 @@
public static final String UPDATE_INTENSIVE = "pregelix.updateIntensive";
/** the check point hook */
public static final String CKP_CLASS = "pregelix.checkpointHook";
+ /** the check point hook */
+ public static final String RECOVERY_COUNT = "pregelix.recoveryCount";
+ /** the checkpoint interval */
+ public static final String CKP_INTERVAL = "pregelix.ckpinterval";
/**
* Construct a Pregelix job from an existing configuration
@@ -223,6 +227,24 @@
getConfiguration().setClass(CKP_CLASS, ckpClass, ICheckpointHook.class);
}
+ /**
+ * Users can provide an ICheckpointHook implementation to specify when to do checkpoint
+ *
+ * @param ckpClass
+ */
+ final public void setRecoveryCount(int recoveryCount) {
+ getConfiguration().setInt(RECOVERY_COUNT, recoveryCount);
+ }
+
+ /**
+ * Users can set the interval of checkpointing
+ *
+ * @param ckpInterval
+ */
+ final public void setCheckpointingInterval(int ckpInterval) {
+ getConfiguration().setInt(CKP_INTERVAL, ckpInterval);
+ }
+
@Override
public String toString() {
return getJobName();
diff --git a/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java b/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
index d68ad2c..4ee1deb 100644
--- a/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
+++ b/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java
@@ -549,4 +549,57 @@
public static String getGlobalAggregateSpillingDirName(Configuration conf, long superStep) {
return "/tmp/pregelix/agg/" + conf.get(PregelixJob.JOB_ID) + "/" + superStep;
}
+
+ /**
+ * Get the path for vertex checkpointing
+ *
+ * @param conf
+ * @param lastSuperStep
+ * @return the path for vertex checkpointing
+ */
+ public static String getVertexCheckpointPath(Configuration conf, long lastSuperStep) {
+ return "/tmp/ckpoint/" + BspUtils.getJobId(conf) + "/vertex/" + lastSuperStep;
+ }
+
+ /**
+ * Get the path for message checkpointing
+ *
+ * @param conf
+ * @param lastSuperStep
+ * @return the path for message checkpointing
+ */
+ public static String getMessageCheckpointPath(Configuration conf, long lastSuperStep) {
+ String path = "/tmp/ckpoint/" + BspUtils.getJobId(conf) + "/message/" + lastSuperStep;
+ return path;
+ }
+
+ /**
+ * Get the path for message checkpointing
+ *
+ * @param conf
+ * @param lastSuperStep
+ * @return the path for message checkpointing
+ */
+ public static String getSecondaryIndexCheckpointPath(Configuration conf, long lastSuperStep) {
+ return "/tmp/ckpoint/" + BspUtils.getJobId(conf) + "/secondaryindex/" + lastSuperStep;
+ }
+
+ /***
+ * Get the recovery count
+ *
+ * @return recovery count
+ */
+ public static int getRecoveryCount(Configuration conf) {
+ return conf.getInt(PregelixJob.RECOVERY_COUNT, 0);
+ }
+
+ /***
+ * Get the user-set checkpoint interval
+ *
+ * @param conf
+ * @return the checkpoint interval
+ */
+ public static int getCheckpointingInterval(Configuration conf) {
+ return conf.getInt(PregelixJob.CKP_INTERVAL, -1);
+ }
}
diff --git a/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/ConservativeCheckpointHook.java b/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/ConservativeCheckpointHook.java
index 6a4a660..4f5fef0 100644
--- a/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/ConservativeCheckpointHook.java
+++ b/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/ConservativeCheckpointHook.java
@@ -25,7 +25,7 @@
@Override
public boolean checkpoint(int superstep) {
- if (superstep % 5 == 0) {
+ if (superstep % 2 == 0) {
return true;
} else {
return false;
diff --git a/pregelix-benchmark/pom.xml b/pregelix-benchmark/pom.xml
new file mode 100644
index 0000000..4d7d456
--- /dev/null
+++ b/pregelix-benchmark/pom.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0"?>
+<project
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+ xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+ <modelVersion>4.0.0</modelVersion>
+ <parent>
+ <artifactId>pregelix</artifactId>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <version>0.2.10-SNAPSHOT</version>
+ </parent>
+
+ <artifactId>pregelix-benchmark</artifactId>
+ <name>pregelix-benchmark</name>
+ <url>http://maven.apache.org</url>
+ <dependencies>
+ <dependency>
+ <groupId>junit</groupId>
+ <artifactId>junit</artifactId>
+ <version>3.8.1</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.giraph</groupId>
+ <artifactId>giraph-core</artifactId>
+ <version>1.0.0</version>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-hdfs-core</artifactId>
+ <version>0.2.10-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/PageRankVertex.java b/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/PageRankVertex.java
new file mode 100644
index 0000000..04c29de
--- /dev/null
+++ b/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/PageRankVertex.java
@@ -0,0 +1,58 @@
+/*
+ * 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.benchmark;
+
+import org.apache.giraph.graph.Vertex;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.VLongWritable;
+
+/**
+ * Demonstrates the basic Pregel PageRank implementation.
+ */
+public class PageRankVertex extends Vertex<VLongWritable, DoubleWritable, FloatWritable, DoubleWritable> {
+
+ public static final String ITERATIONS = "HyracksPageRankVertex.iteration";
+ private final DoubleWritable vertexValue = new DoubleWritable();
+ private final DoubleWritable msg = new DoubleWritable();
+ private int maxIteration = -1;
+
+ @Override
+ public void compute(Iterable<DoubleWritable> msgIterator) {
+ if (maxIteration < 0) {
+ maxIteration = getContext().getConfiguration().getInt(ITERATIONS, 10);
+ }
+ if (getSuperstep() == 1) {
+ vertexValue.set(1.0 / getTotalNumVertices());
+ }
+ if (getSuperstep() >= 2 && getSuperstep() <= maxIteration) {
+ double sum = 0;
+ for (DoubleWritable msg : msgIterator) {
+ sum += msg.get();
+ }
+ vertexValue.set((0.15 / getTotalNumVertices()) + 0.85 * sum);
+ }
+
+ if (getSuperstep() >= 1 && getSuperstep() < maxIteration) {
+ long edges = getNumEdges();
+ msg.set(vertexValue.get() / edges);
+ sendMessageToAllEdges(msg);
+ } else {
+ voteToHalt();
+ }
+ }
+
+}
diff --git a/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java b/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
new file mode 100644
index 0000000..3d85f66
--- /dev/null
+++ b/pregelix-benchmark/src/main/java/edu/uci/ics/pregelix/benchmark/TextPageRankInputFormat.java
@@ -0,0 +1,70 @@
+/*
+ * 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.benchmark;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.giraph.edge.Edge;
+import org.apache.giraph.edge.MapMutableEdge;
+import org.apache.giraph.io.formats.TextVertexInputFormat;
+import org.apache.hadoop.io.DoubleWritable;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.VLongWritable;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+public class TextPageRankInputFormat extends TextVertexInputFormat<VLongWritable, DoubleWritable, FloatWritable> {
+
+ @Override
+ public TextVertexReader createVertexReader(InputSplit split, TaskAttemptContext context) throws IOException {
+ return new TextVertexReaderFromEachLine() {
+ String[] items;
+
+ @Override
+ protected VLongWritable getId(Text line) throws IOException {
+ items = line.toString().split(" ");
+ return new VLongWritable(Long.parseLong(items[0]));
+ }
+
+ @Override
+ protected DoubleWritable getValue(Text line) throws IOException {
+ return null;
+ }
+
+ @Override
+ protected Iterable<Edge<VLongWritable, FloatWritable>> getEdges(Text line) throws IOException {
+ List<Edge<VLongWritable, FloatWritable>> edges = new ArrayList<Edge<VLongWritable, FloatWritable>>();
+ Map<VLongWritable, FloatWritable> edgeMap = new HashMap<VLongWritable, FloatWritable>();
+ for (int i = 1; i < items.length; i++) {
+ edgeMap.put(new VLongWritable(Long.parseLong(items[i])), null);
+ }
+ for (Entry<VLongWritable, FloatWritable> entry : edgeMap.entrySet()) {
+ MapMutableEdge<VLongWritable, FloatWritable> edge = new MapMutableEdge<VLongWritable, FloatWritable>();
+ edge.setEntry(entry);
+ edge.setValue(null);
+ edges.add(edge);
+ }
+ return edges;
+ }
+
+ };
+ }
+}
diff --git a/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
index 7bd2cf8..d6a6f3d 100644
--- a/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
+++ b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/driver/Driver.java
@@ -23,7 +23,9 @@
import java.util.ArrayList;
import java.util.Collections;
import java.util.EnumSet;
+import java.util.HashSet;
import java.util.List;
+import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -50,6 +52,7 @@
import edu.uci.ics.pregelix.core.jobgen.JobGenOuterJoinSingleSort;
import edu.uci.ics.pregelix.core.jobgen.JobGenOuterJoinSort;
import edu.uci.ics.pregelix.core.jobgen.clusterconfig.ClusterConfig;
+import edu.uci.ics.pregelix.core.util.ExceptionUtilities;
import edu.uci.ics.pregelix.dataflow.util.IterationUtils;
@SuppressWarnings("rawtypes")
@@ -89,6 +92,7 @@
this.profiling = profiling;
PregelixJob currentJob = jobs.get(0);
PregelixJob lastJob = currentJob;
+ addHadoopConfiguration(currentJob, ipAddress, port, true);
JobGen jobGen = null;
/** prepare job -- deploy jars */
@@ -99,27 +103,29 @@
IntWritable lastSnapshotSuperstep = new IntWritable(0);
boolean failed = false;
int retryCount = 0;
- int maxRetryCount = 1;
+ int maxRetryCount = 3;
+ jobGen = selectJobGen(planChoice, currentJob);
do {
try {
for (int i = lastSnapshotJobIndex.get(); i < jobs.size(); i++) {
lastJob = currentJob;
currentJob = jobs.get(i);
+ currentJob.setRecoveryCount(retryCount);
/** add hadoop configurations */
- addHadoopConfiguration(currentJob, ipAddress, port);
+ addHadoopConfiguration(currentJob, ipAddress, port, failed);
ICheckpointHook ckpHook = BspUtils.createCheckpointHook(currentJob.getConfiguration());
/** load the data */
- if (i == 0 || compatible(lastJob, currentJob)) {
+ if ((i == 0 || compatible(lastJob, currentJob)) && !failed) {
if (i != 0) {
finishJobs(jobGen, deploymentId);
/** invalidate/clear checkpoint */
lastSnapshotJobIndex.set(0);
lastSnapshotSuperstep.set(0);
}
- jobGen = selectJobGen(planChoice, currentJob);
+ jobGen.reset(currentJob);
loadData(currentJob, jobGen, deploymentId);
} else {
jobGen.reset(currentJob);
@@ -137,12 +143,16 @@
/** clear checkpoints if any */
jobGen.clearCheckpoints();
hcc.unDeployBinary(deploymentId);
- } catch (IOException ioe) {
- /** disk failures */
- //restart from snapshot
- failed = true;
- retryCount++;
- throw new HyracksException(ioe);
+ } catch (Exception e1) {
+ Set<String> blackListNodes = new HashSet<String>();
+ /** disk failures or node failures */
+ if (ExceptionUtilities.recoverable(e1, blackListNodes)) {
+ ClusterConfig.addToBlackListNodes(blackListNodes);
+ failed = true;
+ retryCount++;
+ } else {
+ throw e1;
+ }
}
} while (failed && retryCount < maxRetryCount);
LOG.info("job finished");
@@ -222,9 +232,9 @@
}
private DeploymentId prepareJobs(String ipAddress, int port) throws Exception {
- if (hcc == null)
+ if (hcc == null) {
hcc = new HyracksConnection(ipAddress, port);
-
+ }
URLClassLoader classLoader = (URLClassLoader) exampleClass.getClassLoader();
List<File> jars = new ArrayList<File>();
URL[] urls = classLoader.getURLs();
@@ -235,7 +245,8 @@
return deploymentId;
}
- private void addHadoopConfiguration(PregelixJob job, String ipAddress, int port) throws HyracksException {
+ private void addHadoopConfiguration(PregelixJob job, String ipAddress, int port, boolean loadClusterConfig)
+ throws HyracksException {
URL hadoopCore = job.getClass().getClassLoader().getResource("core-site.xml");
if (hadoopCore != null) {
job.getConfiguration().addResource(hadoopCore);
@@ -248,7 +259,9 @@
if (hadoopHdfs != null) {
job.getConfiguration().addResource(hadoopHdfs);
}
- ClusterConfig.loadClusterConfig(ipAddress, port);
+ if (loadClusterConfig) {
+ ClusterConfig.loadClusterConfig(ipAddress, port);
+ }
}
private void runLoopBody(DeploymentId deploymentId, PregelixJob job, JobGen jobGen, int currentJobIndex,
@@ -256,10 +269,16 @@
throws Exception {
if (doRecovery) {
/** reload the checkpoint */
- runLoadCheckpoint(deploymentId, jobGen, snapshotSuperstep.get());
-
+ if (snapshotSuperstep.get() > 0) {
+ runClearState(deploymentId, jobGen);
+ runLoadCheckpoint(deploymentId, jobGen, snapshotSuperstep.get());
+ } else {
+ runClearState(deploymentId, jobGen);
+ loadData(job, jobGen, deploymentId);
+ }
}
int i = doRecovery ? snapshotSuperstep.get() + 1 : 1;
+ int ckpInterval = BspUtils.getCheckpointingInterval(job.getConfiguration());
boolean terminate = false;
long start, end, time;
do {
@@ -270,10 +289,10 @@
LOG.info(job + ": iteration " + i + " finished " + time + "ms");
terminate = IterationUtils.readTerminationState(job.getConfiguration(), jobGen.getJobId())
|| IterationUtils.readForceTerminationState(job.getConfiguration(), jobGen.getJobId());
- if (ckpHook.checkpoint(i)) {
+ if (ckpHook.checkpoint(i) || (ckpInterval > 0 && i % ckpInterval == 0)) {
runCheckpoint(deploymentId, jobGen, i);
- snapshotSuperstep.set(i);
snapshotJobIndex.set(currentJobIndex);
+ snapshotSuperstep.set(i);
}
i++;
} while (!terminate);
diff --git a/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
index 36723a6..a577bdb 100644
--- a/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
+++ b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java
@@ -25,7 +25,9 @@
import java.io.IOException;
import java.lang.reflect.Type;
import java.util.ArrayList;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
import java.util.UUID;
import java.util.logging.Logger;
@@ -36,9 +38,12 @@
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.mapreduce.InputFormat;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -61,10 +66,7 @@
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.hdfs.api.ITupleWriterFactory;
-import edu.uci.ics.hyracks.hdfs2.dataflow.ConfFactory;
import edu.uci.ics.hyracks.hdfs2.dataflow.HDFSReadOperatorDescriptor;
-import edu.uci.ics.hyracks.hdfs2.dataflow.HDFSWriteOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
@@ -74,7 +76,7 @@
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ConstantMergePolicyFactory;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpOperationTrackerProvider;
import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousSchedulerProvider;
@@ -102,10 +104,10 @@
import edu.uci.ics.pregelix.dataflow.EmptyTupleSourceOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.HDFSFileWriteOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.KeyValueParserFactory;
-import edu.uci.ics.pregelix.dataflow.KeyValueWriterFactory;
import edu.uci.ics.pregelix.dataflow.MaterializingReadOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.MaterializingWriteOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.VertexFileScanOperatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.VertexFileWriteOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.VertexWriteOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
import edu.uci.ics.pregelix.dataflow.std.RuntimeHookOperatorDescriptor;
@@ -134,11 +136,15 @@
protected int frameSize = ClusterConfig.getFrameSize();
protected int maxFrameNumber = (int) (((long) 32 * MB) / frameSize);
+ private static final Map<String, String> MERGE_POLICY_PROPERTIES;
+ static {
+ MERGE_POLICY_PROPERTIES = new HashMap<String, String>();
+ MERGE_POLICY_PROPERTIES.put("num-components", "3");
+ }
+
protected static final String SECONDARY_INDEX_ODD = "secondary1";
protected static final String SECONDARY_INDEX_EVEN = "secondary2";
- private String vertexCheckpointPath;
-
public JobGen(PregelixJob job) {
init(job);
}
@@ -148,10 +154,7 @@
pregelixJob = job;
initJobConfiguration();
job.setJobId(jobId);
-
- vertexCheckpointPath = "/tmp/ckpoint/" + jobId + "/vertex";
- // set the frame size to be the one user specified if the user did
- // specify.
+ // set the frame size to be the one user specified if the user did specify.
int specifiedFrameSize = BspUtils.getFrameSize(job.getConfiguration());
if (specifiedFrameSize > 0) {
frameSize = specifiedFrameSize;
@@ -376,15 +379,21 @@
@Override
public JobSpecification[] generateCheckpointing(int lastSuccessfulIteration) throws HyracksException {
try {
-
PregelixJob tmpJob = this.createCloneJob("Vertex checkpointing for job " + jobId, pregelixJob);
tmpJob.setVertexOutputFormatClass(InternalVertexOutputFormat.class);
- FileOutputFormat.setOutputPath(tmpJob, new Path(vertexCheckpointPath + "/" + lastSuccessfulIteration));
+ FileOutputFormat.setOutputPath(tmpJob,
+ new Path(BspUtils.getVertexCheckpointPath(conf, lastSuccessfulIteration)));
tmpJob.setOutputKeyClass(NullWritable.class);
tmpJob.setOutputValueClass(BspUtils.getVertexClass(tmpJob.getConfiguration()));
+ FileSystem dfs = FileSystem.get(tmpJob.getConfiguration());
+
+ dfs.delete(new Path(BspUtils.getVertexCheckpointPath(conf, lastSuccessfulIteration)), true);
JobSpecification vertexCkpSpec = scanIndexWriteToHDFS(tmpJob.getConfiguration());
+
+ dfs.delete(new Path(BspUtils.getMessageCheckpointPath(conf, lastSuccessfulIteration)), true);
JobSpecification[] stateCkpSpecs = generateStateCheckpointing(lastSuccessfulIteration);
JobSpecification[] specs = new JobSpecification[1 + stateCkpSpecs.length];
+
specs[0] = vertexCkpSpec;
for (int i = 1; i < specs.length; i++) {
specs[i] = stateCkpSpecs[i - 1];
@@ -397,7 +406,7 @@
@Override
public JobSpecification generateLoadingJob() throws HyracksException {
- JobSpecification spec = loadHDFSData(conf);
+ JobSpecification spec = loadHDFSData(pregelixJob);
return spec;
}
@@ -412,13 +421,22 @@
try {
PregelixJob tmpJob = this.createCloneJob("Vertex checkpoint loading for job " + jobId, pregelixJob);
tmpJob.setVertexInputFormatClass(InternalVertexInputFormat.class);
- FileInputFormat.setInputPaths(tmpJob, new Path(vertexCheckpointPath + "/" + lastCheckpointedIteration));
- JobSpecification vertexLoadSpec = loadHDFSData(tmpJob.getConfiguration());
+ FileInputFormat.setInputPaths(tmpJob,
+ new Path(BspUtils.getVertexCheckpointPath(conf, lastCheckpointedIteration)));
+ JobSpecification[] cleanVertices = generateCleanup();
+ JobSpecification createIndex = generateCreatingJob();
+ JobSpecification vertexLoadSpec = loadHDFSData(tmpJob);
JobSpecification[] stateLoadSpecs = generateStateCheckpointLoading(lastCheckpointedIteration, tmpJob);
- JobSpecification[] specs = new JobSpecification[1 + stateLoadSpecs.length];
- specs[0] = vertexLoadSpec;
- for (int i = 1; i < specs.length; i++) {
- specs[i] = stateLoadSpecs[i - 1];
+ JobSpecification[] specs = new JobSpecification[cleanVertices.length + 2 + stateLoadSpecs.length];
+
+ int i = 0;
+ for (; i < cleanVertices.length; i++) {
+ specs[i] = cleanVertices[i];
+ }
+ specs[i++] = createIndex;
+ specs[i++] = vertexLoadSpec;
+ for (; i < specs.length; i++) {
+ specs[i] = stateLoadSpecs[i - cleanVertices.length - 2];
}
return specs;
} catch (Exception e) {
@@ -470,16 +488,17 @@
protected IIndexDataflowHelperFactory getIndexDataflowHelperFactory() {
if (BspUtils.useLSM(conf)) {
- return new LSMBTreeDataflowHelperFactory(new VirtualBufferCacheProvider(), new ConstantMergePolicyProvider(
- 3), NoOpOperationTrackerProvider.INSTANCE, SynchronousSchedulerProvider.INSTANCE,
- NoOpIOOperationCallback.INSTANCE, 0.01);
+ return new LSMBTreeDataflowHelperFactory(new VirtualBufferCacheProvider(),
+ new ConstantMergePolicyFactory(), MERGE_POLICY_PROPERTIES, NoOpOperationTrackerProvider.INSTANCE,
+ SynchronousSchedulerProvider.INSTANCE, NoOpIOOperationCallback.INSTANCE, 0.01);
} else {
return new BTreeDataflowHelperFactory();
}
}
@SuppressWarnings({ "rawtypes", "unchecked" })
- private JobSpecification loadHDFSData(Configuration conf) throws HyracksException, HyracksDataException {
+ private JobSpecification loadHDFSData(PregelixJob job) throws HyracksException, HyracksDataException {
+ Configuration conf = job.getConfiguration();
Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(conf);
Class<? extends Writable> vertexClass = BspUtils.getVertexClass(conf);
JobSpecification spec = new JobSpecification();
@@ -492,7 +511,7 @@
VertexInputFormat inputFormat = BspUtils.createVertexInputFormat(conf);
List<InputSplit> splits = new ArrayList<InputSplit>();
try {
- splits = inputFormat.getSplits(pregelixJob, fileSplitProvider.getFileSplits().length);
+ splits = inputFormat.getSplits(job, fileSplitProvider.getFileSplits().length);
LOGGER.info("number of splits: " + splits.size());
for (InputSplit split : splits)
LOGGER.info(split.toString());
@@ -591,7 +610,8 @@
*/
IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
vertexIdClass.getName(), vertexClass.getName());
- HDFSFileWriteOperatorDescriptor writer = new HDFSFileWriteOperatorDescriptor(spec, confFactory, inputRdFactory);
+ VertexFileWriteOperatorDescriptor writer = new VertexFileWriteOperatorDescriptor(spec, confFactory,
+ inputRdFactory);
ClusterConfig.setLocationConstraint(spec, writer);
/**
@@ -637,18 +657,21 @@
/**
* construct the materializing write operator
*/
- MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal);
+ MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal,
+ false);
ClusterConfig.setLocationConstraint(spec, materializeRead);
- String checkpointPath = "/tmp/ckpoint/" + jobId + "/message/" + lastSuccessfulIteration;
+ String checkpointPath = BspUtils.getMessageCheckpointPath(conf, lastSuccessfulIteration);;
PregelixJob tmpJob = createCloneJob("State checkpointing for job " + jobId, pregelixJob);
- tmpJob.setVertexOutputFormatClass(InternalVertexOutputFormat.class);
+ tmpJob.setOutputFormatClass(SequenceFileOutputFormat.class);
FileOutputFormat.setOutputPath(tmpJob, new Path(checkpointPath));
tmpJob.setOutputKeyClass(vertexIdClass);
tmpJob.setOutputValueClass(MsgList.class);
- ITupleWriterFactory writerFactory = new KeyValueWriterFactory(new ConfFactory(tmpJob));
- HDFSWriteOperatorDescriptor hdfsWriter = new HDFSWriteOperatorDescriptor(spec, tmpJob, writerFactory);
+ IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
+ vertexIdClass.getName(), MsgList.class.getName());
+ HDFSFileWriteOperatorDescriptor hdfsWriter = new HDFSFileWriteOperatorDescriptor(spec, tmpJob, inputRdFactory);
+ ClusterConfig.setLocationConstraint(spec, hdfsWriter);
spec.connect(new OneToOneConnectorDescriptor(spec), emptyTupleSource, 0, materializeRead, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), materializeRead, 0, hdfsWriter, 0);
@@ -660,25 +683,26 @@
@SuppressWarnings({ "unchecked", "rawtypes" })
protected JobSpecification[] generateStateCheckpointLoading(int lastCheckpointedIteration, PregelixJob job)
throws HyracksException {
- String checkpointPath = "/tmp/ckpoint/" + jobId + "/message/" + lastCheckpointedIteration;
+ String checkpointPath = BspUtils.getMessageCheckpointPath(job.getConfiguration(), lastCheckpointedIteration);
PregelixJob tmpJob = createCloneJob("State checkpoint loading for job " + jobId, job);
- tmpJob.setVertexInputFormatClass(InternalVertexInputFormat.class);
+ tmpJob.setInputFormatClass(SequenceFileInputFormat.class);
try {
FileInputFormat.setInputPaths(tmpJob, checkpointPath);
} catch (IOException e) {
throw new HyracksException(e);
}
- Configuration conf = job.getConfiguration();
+ Configuration conf = tmpJob.getConfiguration();
Class vertexIdClass = BspUtils.getVertexIndexClass(conf);
JobSpecification spec = new JobSpecification();
/***
* HDFS read operator
*/
- VertexInputFormat inputFormat = BspUtils.createVertexInputFormat(conf);
List<InputSplit> splits = new ArrayList<InputSplit>();
try {
- splits = inputFormat.getSplits(tmpJob, ClusterConfig.getLocationConstraint().length);
+ InputFormat inputFormat = org.apache.hadoop.util.ReflectionUtils.newInstance(job.getInputFormatClass(),
+ job.getConfiguration());
+ splits = inputFormat.getSplits(tmpJob);
LOGGER.info("number of splits: " + splits.size());
for (InputSplit split : splits)
LOGGER.info(split.toString());
@@ -692,6 +716,16 @@
readSchedule, new KeyValueParserFactory());
ClusterConfig.setLocationConstraint(spec, scanner);
+ /** construct the sort operator to sort message states */
+ int[] keyFields = new int[] { 0 };
+ INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
+ IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
+ sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(lastCheckpointedIteration,
+ WritableComparator.get(vertexIdClass).getClass());
+ ExternalSortOperatorDescriptor sort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
+ nkmFactory, sortCmpFactories, recordDescriptor);
+ ClusterConfig.setLocationConstraint(spec, sort);
+
/**
* construct the materializing write operator
*/
@@ -701,7 +735,7 @@
/** construct runtime hook */
RuntimeHookOperatorDescriptor postSuperStep = new RuntimeHookOperatorDescriptor(spec,
- new RecoveryRuntimeHookFactory(jobId, lastCheckpointedIteration + 1, new ConfigurationFactory(
+ new RecoveryRuntimeHookFactory(jobId, lastCheckpointedIteration, new ConfigurationFactory(
pregelixJob.getConfiguration())));
ClusterConfig.setLocationConstraint(spec, postSuperStep);
@@ -713,8 +747,8 @@
* connect operator descriptors
*/
ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 0,
- materialize, 0);
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 0, sort, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sort, 0, materialize, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), materialize, 0, postSuperStep, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), postSuperStep, 0, emptySink, 0);
spec.setFrameSize(frameSize);
diff --git a/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
index 41887c0..9389f62 100644
--- a/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
+++ b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java
@@ -21,15 +21,17 @@
import java.util.List;
import java.util.logging.Logger;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.VLongWritable;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.io.WritableComparator;
+import org.apache.hadoop.mapreduce.InputFormat;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
+import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat;
import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -49,17 +51,12 @@
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.hdfs2.dataflow.ConfFactory;
import edu.uci.ics.hyracks.hdfs2.dataflow.HDFSReadOperatorDescriptor;
-import edu.uci.ics.hyracks.hdfs2.dataflow.HDFSWriteOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
import edu.uci.ics.pregelix.api.graph.MsgList;
-import edu.uci.ics.pregelix.api.io.VertexInputFormat;
-import edu.uci.ics.pregelix.api.io.internal.InternalVertexInputFormat;
-import edu.uci.ics.pregelix.api.io.internal.InternalVertexOutputFormat;
import edu.uci.ics.pregelix.api.job.PregelixJob;
import edu.uci.ics.pregelix.api.util.BspUtils;
import edu.uci.ics.pregelix.core.data.TypeTraits;
@@ -71,8 +68,8 @@
import edu.uci.ics.pregelix.dataflow.EmptySinkOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.EmptyTupleSourceOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.FinalAggregateOperatorDescriptor;
+import edu.uci.ics.pregelix.dataflow.HDFSFileWriteOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.KeyValueParserFactory;
-import edu.uci.ics.pregelix.dataflow.KeyValueWriterFactory;
import edu.uci.ics.pregelix.dataflow.MaterializingReadOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.MaterializingWriteOperatorDescriptor;
import edu.uci.ics.pregelix.dataflow.TerminationStateWriterOperatorDescriptor;
@@ -344,7 +341,8 @@
/**
* construct the materializing write operator
*/
- MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal);
+ MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal,
+ true);
ClusterConfig.setLocationConstraint(spec, materializeRead);
/**
@@ -525,14 +523,10 @@
/** generate plan specific state checkpointing */
protected JobSpecification[] generateStateCheckpointing(int lastSuccessfulIteration) throws HyracksException {
JobSpecification[] msgCkpSpecs = super.generateStateCheckpointing(lastSuccessfulIteration);
- PregelixJob tmpJob = this.createCloneJob("Vertex checkpointing for job " + jobId, pregelixJob);
- tmpJob.setVertexOutputFormatClass(InternalVertexOutputFormat.class);
/** generate secondary index checkpoint */
- String checkpointPath = "/tmp/ckpoint/" + jobId + "/secondary/" + lastSuccessfulIteration;
- FileOutputFormat.setOutputPath(tmpJob, new Path(checkpointPath));
- tmpJob.setOutputKeyClass(BspUtils.getVertexIndexClass(tmpJob.getConfiguration()));
- tmpJob.setOutputValueClass(MsgList.class);
+ PregelixJob tmpJob = this.createCloneJob("Secondary index checkpointing for job " + jobId, pregelixJob);
+
JobSpecification secondaryBTreeCkp = generateSecondaryBTreeCheckpoint(lastSuccessfulIteration, tmpJob);
JobSpecification[] specs = new JobSpecification[msgCkpSpecs.length + 1];
@@ -549,11 +543,12 @@
@Override
protected JobSpecification[] generateStateCheckpointLoading(int lastSuccessfulIteration, PregelixJob job)
throws HyracksException {
- JobSpecification[] msgCkpSpecs = generateStateCheckpointLoading(lastSuccessfulIteration, job);
- PregelixJob tmpJob = this.createCloneJob("Vertex checkpointing for job " + jobId, pregelixJob);
- tmpJob.setVertexOutputFormatClass(InternalVertexOutputFormat.class);
+ /** generate message checkpoint load */
+ JobSpecification[] msgCkpSpecs = super.generateStateCheckpointLoading(lastSuccessfulIteration, job);
/** generate secondary index checkpoint load */
+ PregelixJob tmpJob = this.createCloneJob("Secondary index checkpoint loading for job " + jobId, pregelixJob);
+ tmpJob.setOutputFormatClass(SequenceFileOutputFormat.class);
JobSpecification secondaryBTreeCkpLoad = generateSecondaryBTreeCheckpointLoad(lastSuccessfulIteration, tmpJob);
JobSpecification[] specs = new JobSpecification[msgCkpSpecs.length + 1];
for (int i = 0; i < msgCkpSpecs.length; i++) {
@@ -569,23 +564,23 @@
Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(job.getConfiguration());
JobSpecification spec = new JobSpecification();
- String checkpointPath = "/tmp/ckpoint/" + jobId + "/secondary/" + lastSuccessfulIteration;;
+ String checkpointPath = BspUtils.getSecondaryIndexCheckpointPath(conf, lastSuccessfulIteration);
PregelixJob tmpJob = createCloneJob("State checkpoint loading for job " + jobId, job);
- tmpJob.setVertexInputFormatClass(InternalVertexInputFormat.class);
+ tmpJob.setInputFormatClass(SequenceFileInputFormat.class);
try {
FileInputFormat.setInputPaths(tmpJob, checkpointPath);
} catch (IOException e) {
throw new HyracksException(e);
}
- Configuration conf = job.getConfiguration();
/***
- * construct HDFS read operator
+ * HDFS read operator
*/
- VertexInputFormat inputFormat = BspUtils.createVertexInputFormat(conf);
List<InputSplit> splits = new ArrayList<InputSplit>();
try {
- splits = inputFormat.getSplits(tmpJob, ClusterConfig.getLocationConstraint().length);
+ InputFormat inputFormat = org.apache.hadoop.util.ReflectionUtils.newInstance(job.getInputFormatClass(),
+ job.getConfiguration());
+ splits = inputFormat.getSplits(tmpJob);
LOGGER.info("number of splits: " + splits.size());
for (InputSplit split : splits)
LOGGER.info(split.toString());
@@ -599,6 +594,16 @@
readSchedule, new KeyValueParserFactory());
ClusterConfig.setLocationConstraint(spec, scanner);
+ /** construct the sort operator to sort message states */
+ int[] keyFields = new int[] { 0 };
+ INormalizedKeyComputerFactory nkmFactory = JobGenUtil.getINormalizedKeyComputerFactory(conf);
+ IBinaryComparatorFactory[] sortCmpFactories = new IBinaryComparatorFactory[1];
+ sortCmpFactories[0] = JobGenUtil.getIBinaryComparatorFactory(lastSuccessfulIteration,
+ WritableComparator.get(vertexIdClass).getClass());
+ ExternalSortOperatorDescriptor sort = new ExternalSortOperatorDescriptor(spec, maxFrameNumber, keyFields,
+ nkmFactory, sortCmpFactories, recordDescriptor);
+ ClusterConfig.setLocationConstraint(spec, sort);
+
/**
* construct bulk-load index operator
*/
@@ -621,8 +626,8 @@
* connect operator descriptors
*/
ITuplePartitionComputerFactory hashPartitionComputerFactory = getVertexPartitionComputerFactory();
- spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 0,
- btreeBulkLoad, 0);
+ spec.connect(new MToNPartitioningConnectorDescriptor(spec, hashPartitionComputerFactory), scanner, 0, sort, 0);
+ spec.connect(new OneToOneConnectorDescriptor(spec), sort, 0, btreeBulkLoad, 0);
spec.setFrameSize(frameSize);
return spec;
@@ -631,6 +636,12 @@
@SuppressWarnings({ "rawtypes", "unchecked" })
private JobSpecification generateSecondaryBTreeCheckpoint(int lastSuccessfulIteration, PregelixJob job)
throws HyracksException {
+ job.setOutputFormatClass(SequenceFileOutputFormat.class);
+ String checkpointPath = BspUtils.getSecondaryIndexCheckpointPath(conf, lastSuccessfulIteration);
+ FileOutputFormat.setOutputPath(job, new Path(checkpointPath));
+ job.setOutputKeyClass(BspUtils.getVertexIndexClass(job.getConfiguration()));
+ job.setOutputValueClass(MsgList.class);
+
Class<? extends WritableComparable<?>> vertexIdClass = BspUtils.getVertexIndexClass(job.getConfiguration());
Class<? extends Writable> msgListClass = MsgList.class;
String readFile = lastSuccessfulIteration % 2 == 0 ? SECONDARY_INDEX_EVEN : SECONDARY_INDEX_ODD;
@@ -654,7 +665,6 @@
/**
* construct btree search operator
*/
- ConfFactory confFactory = new ConfFactory(job);
RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(
vertexIdClass.getName(), msgListClass.getName());
IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1];
@@ -674,8 +684,9 @@
/**
* construct write file operator
*/
- HDFSWriteOperatorDescriptor writer = new HDFSWriteOperatorDescriptor(spec, job, new KeyValueWriterFactory(
- confFactory));
+ IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses(
+ vertexIdClass.getName(), MsgList.class.getName());
+ HDFSFileWriteOperatorDescriptor writer = new HDFSFileWriteOperatorDescriptor(spec, job, inputRdFactory);
ClusterConfig.setLocationConstraint(spec, writer);
/**
diff --git a/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
index c29ea18..287b797 100644
--- a/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
+++ b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java
@@ -299,7 +299,8 @@
/**
* construct the materializing write operator
*/
- MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal);
+ MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal,
+ true);
ClusterConfig.setLocationConstraint(spec, materializeRead);
/**
diff --git a/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
index dc61971..3b3c9e7 100644
--- a/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
+++ b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java
@@ -288,7 +288,8 @@
/**
* construct the materializing write operator
*/
- MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal);
+ MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal,
+ true);
ClusterConfig.setLocationConstraint(spec, materializeRead);
/**
diff --git a/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
index 34f723f..e334095 100644
--- a/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
+++ b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java
@@ -302,7 +302,8 @@
/**
* construct the materializing write operator
*/
- MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal);
+ MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal,
+ true);
ClusterConfig.setLocationConstraint(spec, materializeRead);
/**
diff --git a/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/clusterconfig/ClusterConfig.java b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/clusterconfig/ClusterConfig.java
index ea6cc8a..89fbdcd 100644
--- a/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/clusterconfig/ClusterConfig.java
+++ b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/clusterconfig/ClusterConfig.java
@@ -20,12 +20,15 @@
import java.net.InetAddress;
import java.util.ArrayList;
import java.util.Arrays;
+import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Random;
+import java.util.Set;
import java.util.TreeMap;
import org.apache.hadoop.mapreduce.InputSplit;
@@ -52,6 +55,8 @@
private static Map<String, List<String>> ipToNcMapping;
private static String[] stores;
private static Scheduler hdfsScheduler;
+ private static Set<String> blackListNodes = new HashSet<String>();
+ private static IHyracksClientConnection hcc;
/**
* let tests set config path to be whatever
@@ -197,9 +202,19 @@
public static void loadClusterConfig(String ipAddress, int port) throws HyracksException {
try {
- IHyracksClientConnection hcc = new HyracksConnection(ipAddress, port);
+ if (hcc == null) {
+ hcc = new HyracksConnection(ipAddress, port);
+ }
Map<String, NodeControllerInfo> ncNameToNcInfos = new TreeMap<String, NodeControllerInfo>();
ncNameToNcInfos.putAll(hcc.getNodeControllerInfos());
+
+ /**
+ * remove black list nodes -- which had disk failures
+ */
+ for (String blackListNode : blackListNodes) {
+ ncNameToNcInfos.remove(blackListNode);
+ }
+
NCs = new String[ncNameToNcInfos.size()];
ipToNcMapping = new HashMap<String, List<String>>();
int i = 0;
@@ -216,7 +231,7 @@
i++;
}
- hdfsScheduler = new Scheduler(ipAddress, port);
+ hdfsScheduler = new Scheduler(hcc.getNodeControllerInfos(), hcc.getClusterTopology());
} catch (Exception e) {
throw new IllegalStateException(e);
}
@@ -240,4 +255,8 @@
}
return locations;
}
+
+ public static void addToBlackListNodes(Collection<String> nodes) {
+ blackListNodes.addAll(nodes);
+ }
}
diff --git a/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/ExceptionUtilities.java b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/ExceptionUtilities.java
new file mode 100644
index 0000000..a4c4501
--- /dev/null
+++ b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/ExceptionUtilities.java
@@ -0,0 +1,88 @@
+/*
+ * 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.core.util;
+
+import java.io.IOException;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * The util to analysis exceptions
+ *
+ * @author yingyib
+ */
+public class ExceptionUtilities {
+
+ /**
+ * Check whether a exception is recoverable or not
+ *
+ * @param exception
+ * @return true or false
+ */
+ public static boolean recoverable(Exception exception, Set<String> blackListNodes) {
+ String message = exception.getMessage();
+
+ /***
+ * check interrupted exception
+ */
+ if (exception instanceof InterruptedException || (message.contains("Node") && message.contains("not live"))
+ || message.contains("Failure occurred on input")) {
+ return true;
+ }
+ Throwable cause = exception;
+ while ((cause = cause.getCause()) != null) {
+ if (cause instanceof InterruptedException) {
+ return true;
+ }
+ }
+
+ /***
+ * check io exception
+ */
+ cause = exception;
+ String blackListNode = null;
+ if (cause instanceof HyracksDataException) {
+ blackListNode = ((HyracksDataException) cause).getNodeId();
+ }
+ while ((cause = cause.getCause()) != null) {
+ if (cause instanceof IOException) {
+ if (containsIOManager(cause)) {
+ if (blackListNode != null) {
+ blackListNodes.add(blackListNode);
+ }
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ /**
+ * Check if the exception traces contains the IOManager, which means there are disk failures
+ *
+ * @param cause
+ * @return true if IOManager is in the trace; false otherwise.
+ */
+ private static boolean containsIOManager(Throwable cause) {
+ StackTraceElement[] traces = cause.getStackTrace();
+ for (StackTraceElement e : traces) {
+ if (e.getClassName().endsWith("IOManager")) {
+ return true;
+ }
+ }
+ return false;
+ }
+}
diff --git a/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
index e1795de..aabd4ba 100644
--- a/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
+++ b/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
@@ -61,8 +61,8 @@
ccConfig.defaultMaxJobAttempts = 0;
ccConfig.jobHistorySize = 1;
ccConfig.profileDumpPeriod = -1;
- //ccConfig.heartbeatPeriod = 5000;
- //ccConfig.maxHeartbeatLapsePeriods = 1;
+ ccConfig.heartbeatPeriod = 50;
+ ccConfig.maxHeartbeatLapsePeriods = 15;
// cluster controller
cc = new ClusterControllerService(ccConfig);
@@ -98,14 +98,22 @@
ClusterConfig.loadClusterConfig(CC_HOST, TEST_HYRACKS_CC_CLIENT_PORT);
}
- public static void showDownNC1() throws Exception {
+ public static void startNC1() throws Exception {
+ nc1.start();
+ }
+
+ public static void shutdownNC1() throws Exception {
nc1.stop();
}
- public static void showDownNC2() throws Exception {
+ public static void shutdownNC2() throws Exception {
nc2.stop();
}
+ public static void shutdownCC() throws Exception {
+ cc.stop();
+ }
+
public static void deinit() throws Exception {
nc2.stop();
nc1.stop();
diff --git a/pregelix-core/src/main/resources/scripts/startcc.sh b/pregelix-core/src/main/resources/scripts/startcc.sh
index 2a6cab2..d7a0ead 100644
--- a/pregelix-core/src/main/resources/scripts/startcc.sh
+++ b/pregelix-core/src/main/resources/scripts/startcc.sh
@@ -58,8 +58,8 @@
if [ -f "conf/topology.xml" ]; then
#Launch hyracks cc script with topology
-${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 -cluster-topology "conf/topology.xml" &> $CCLOGS_DIR/cc.log &
+${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -heartbeat-period 5000 -max-heartbeat-lapse-periods 4 -default-max-job-attempts 0 -job-history-size 0 -cluster-topology "conf/topology.xml" &> $CCLOGS_DIR/cc.log &
else
#Launch hyracks cc script without toplogy
-${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 &> $CCLOGS_DIR/cc.log &
+${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -heartbeat-period 5000 -max-heartbeat-lapse-periods 4 -default-max-job-attempts 0 -job-history-size 0 &> $CCLOGS_DIR/cc.log &
fi
diff --git a/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java b/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
index 7221cb5..b22e468 100644
--- a/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
+++ b/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java
@@ -230,8 +230,16 @@
@Override
public void fail() throws HyracksDataException {
- for (IFrameWriter writer : writers)
+ try {
+ cursor.close();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ treeIndexOpHelper.close();
+ }
+ for (IFrameWriter writer : writers) {
writer.fail();
+ }
}
@Override
diff --git a/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java b/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
index b21cd2a..0ecfd03 100644
--- a/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
+++ b/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java
@@ -254,8 +254,16 @@
@Override
public void fail() throws HyracksDataException {
- for (IFrameWriter writer : writers)
+ try {
+ cursor.close();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ treeIndexOpHelper.close();
+ }
+ for (IFrameWriter writer : writers) {
writer.fail();
+ }
}
/** compare tuples */
diff --git a/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java b/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java
index dd6ee3c..e64e9cc 100644
--- a/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java
+++ b/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java
@@ -245,6 +245,13 @@
@Override
public void fail() throws HyracksDataException {
+ try {
+ cursor.close();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ treeIndexOpHelper.close();
+ }
writer.fail();
}
diff --git a/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java b/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
index 3cebfb8..a9c787f 100644
--- a/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
+++ b/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java
@@ -219,8 +219,16 @@
@Override
public void fail() throws HyracksDataException {
- for (IFrameWriter writer : writers)
+ try {
+ cursor.close();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ treeIndexOpHelper.close();
+ }
+ for (IFrameWriter writer : writers) {
writer.fail();
+ }
}
/** compare tuples */
diff --git a/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java b/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java
index bbe2764..86a211f 100644
--- a/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java
+++ b/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java
@@ -205,6 +205,13 @@
@Override
public void fail() throws HyracksDataException {
+ try {
+ cursor.close();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ treeIndexOpHelper.close();
+ }
writer.fail();
}
diff --git a/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java b/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java
index c4890e1..c985f64 100644
--- a/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java
+++ b/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeIndexBulkReLoadOperatorNodePushable.java
@@ -100,6 +100,11 @@
@Override
public void fail() throws HyracksDataException {
-
+ try {
+ bulkLoader.end();
+ } catch (IndexException e) {
+ treeIndexOpHelper.close();
+ throw new HyracksDataException(e);
+ }
}
}
\ No newline at end of file
diff --git a/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java b/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java
index bd85e3e..de87909 100644
--- a/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java
+++ b/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java
@@ -224,8 +224,16 @@
@Override
public void fail() throws HyracksDataException {
- for (IFrameWriter writer : writers)
+ try {
+ cursor.close();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ treeIndexHelper.close();
+ }
+ for (IFrameWriter writer : writers) {
writer.fail();
+ }
}
@Override
diff --git a/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java b/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java
index 9a21680..a1177c8 100644
--- a/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java
+++ b/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/HDFSFileWriteOperatorDescriptor.java
@@ -19,42 +19,45 @@
import java.io.IOException;
import java.nio.ByteBuffer;
-import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.RecordWriter;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.ReflectionUtils;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameDeserializer;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
import edu.uci.ics.hyracks.hdfs.ContextFactory;
-import edu.uci.ics.pregelix.api.graph.Vertex;
-import edu.uci.ics.pregelix.api.io.VertexOutputFormat;
-import edu.uci.ics.pregelix.api.io.VertexWriter;
-import edu.uci.ics.pregelix.api.util.BspUtils;
-import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
+import edu.uci.ics.hyracks.hdfs2.dataflow.ConfFactory;
import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
public class HDFSFileWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
private static final long serialVersionUID = 1L;
- private final IConfigurationFactory confFactory;
+ private final ConfFactory confFactory;
private final IRecordDescriptorFactory inputRdFactory;
- public HDFSFileWriteOperatorDescriptor(JobSpecification spec, IConfigurationFactory confFactory,
- IRecordDescriptorFactory inputRdFactory) {
+ public HDFSFileWriteOperatorDescriptor(JobSpecification spec, Job conf, IRecordDescriptorFactory inputRdFactory)
+ throws HyracksException {
super(spec, 1, 0);
- this.confFactory = confFactory;
- this.inputRdFactory = inputRdFactory;
+ try {
+ this.confFactory = new ConfFactory(conf);
+ this.inputRdFactory = inputRdFactory;
+ } catch (Exception e) {
+ throw new HyracksException(e);
+ }
}
@SuppressWarnings("rawtypes")
@@ -65,12 +68,12 @@
return new AbstractUnaryInputSinkOperatorNodePushable() {
private RecordDescriptor rd0;
private FrameDeserializer frameDeserializer;
- private Configuration conf;
- private VertexWriter vertexWriter;
+ private Job job;
+ private RecordWriter recordWriter;
private TaskAttemptContext context;
+ private ContextFactory ctxFactory = new ContextFactory();
private String TEMP_DIR = "_temporary";
private ClassLoader ctxCL;
- private ContextFactory ctxFactory = new ContextFactory();
@Override
public void open() throws HyracksDataException {
@@ -79,16 +82,16 @@
frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), rd0);
ctxCL = Thread.currentThread().getContextClassLoader();
Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
- conf = confFactory.createConfiguration(ctx);
-
- VertexOutputFormat outputFormat = BspUtils.createVertexOutputFormat(conf);
- context = ctxFactory.createContext(conf, partition);
- context.getConfiguration().setClassLoader(ctx.getJobletContext().getClassLoader());
+ job = confFactory.getConf();
try {
- vertexWriter = outputFormat.createVertexWriter(context);
+ OutputFormat outputFormat = ReflectionUtils.newInstance(job.getOutputFormatClass(),
+ job.getConfiguration());
+ context = ctxFactory.createContext(job.getConfiguration(), partition);
+ context.getConfiguration().setClassLoader(ctx.getJobletContext().getClassLoader());
+ recordWriter = outputFormat.getRecordWriter(context);
} catch (InterruptedException e) {
throw new HyracksDataException(e);
- } catch (IOException e) {
+ } catch (Exception e) {
throw new HyracksDataException(e);
}
}
@@ -100,8 +103,9 @@
try {
while (!frameDeserializer.done()) {
Object[] tuple = frameDeserializer.deserializeRecord();
- Vertex value = (Vertex) tuple[1];
- vertexWriter.writeVertex(value);
+ Object key = tuple[0];
+ Object value = tuple[1];
+ recordWriter.write(key, value);
}
} catch (InterruptedException e) {
throw new HyracksDataException(e);
@@ -118,7 +122,7 @@
@Override
public void close() throws HyracksDataException {
try {
- vertexWriter.close(context);
+ recordWriter.close(context);
moveFilesToFinalPath();
} catch (InterruptedException e) {
throw new HyracksDataException(e);
@@ -129,9 +133,8 @@
private void moveFilesToFinalPath() throws HyracksDataException {
try {
- JobContext job = ctxFactory.createJobContext(conf);
Path outputPath = FileOutputFormat.getOutputPath(job);
- FileSystem dfs = FileSystem.get(conf);
+ FileSystem dfs = FileSystem.get(job.getConfiguration());
Path filePath = new Path(outputPath, "part-" + new Integer(partition).toString());
FileStatus[] results = findPartitionPaths(outputPath, dfs);
if (results.length >= 1) {
@@ -161,7 +164,7 @@
FileStatus[] tempPaths = dfs.listStatus(outputPath, new PathFilter() {
@Override
public boolean accept(Path dir) {
- return dir.getName().endsWith(TEMP_DIR);
+ return dir.getName().endsWith(TEMP_DIR) && dir.getName().indexOf(".crc") < 0;
}
});
Path tempDir = tempPaths[0].getPath();
diff --git a/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java b/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
index ca8f190..b44b643 100644
--- a/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
+++ b/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java
@@ -30,9 +30,12 @@
public class MaterializingReadOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
private static final long serialVersionUID = 1L;
+ private final boolean removeIterationState;
- public MaterializingReadOperatorDescriptor(JobSpecification spec, RecordDescriptor recordDescriptor) {
+ public MaterializingReadOperatorDescriptor(JobSpecification spec, RecordDescriptor recordDescriptor,
+ boolean removeIterationState) {
super(spec, 1, 1);
+ this.removeIterationState = removeIterationState;
recordDescriptors[0] = recordDescriptor;
}
@@ -73,7 +76,7 @@
@Override
public void fail() throws HyracksDataException {
-
+ writer.fail();
}
@Override
@@ -81,7 +84,9 @@
/**
* remove last iteration's state
*/
- IterationUtils.removeIterationState(ctx, partition);
+ if (removeIterationState) {
+ IterationUtils.removeIterationState(ctx, partition);
+ }
writer.close();
complete = true;
}
diff --git a/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileWriteOperatorDescriptor.java b/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileWriteOperatorDescriptor.java
new file mode 100644
index 0000000..f3ec40e
--- /dev/null
+++ b/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileWriteOperatorDescriptor.java
@@ -0,0 +1,192 @@
+/*
+ * 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.dataflow;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameDeserializer;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import edu.uci.ics.hyracks.hdfs.ContextFactory;
+import edu.uci.ics.pregelix.api.graph.Vertex;
+import edu.uci.ics.pregelix.api.io.VertexOutputFormat;
+import edu.uci.ics.pregelix.api.io.VertexWriter;
+import edu.uci.ics.pregelix.api.util.BspUtils;
+import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory;
+import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory;
+
+public class VertexFileWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+ private static final long serialVersionUID = 1L;
+ private final IConfigurationFactory confFactory;
+ private final IRecordDescriptorFactory inputRdFactory;
+
+ public VertexFileWriteOperatorDescriptor(JobSpecification spec, IConfigurationFactory confFactory,
+ IRecordDescriptorFactory inputRdFactory) {
+ super(spec, 1, 0);
+ this.confFactory = confFactory;
+ this.inputRdFactory = inputRdFactory;
+ }
+
+ @SuppressWarnings("rawtypes")
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+ throws HyracksDataException {
+ return new AbstractUnaryInputSinkOperatorNodePushable() {
+ private RecordDescriptor rd0;
+ private FrameDeserializer frameDeserializer;
+ private Configuration conf;
+ private VertexWriter vertexWriter;
+ private TaskAttemptContext context;
+ private String TEMP_DIR = "_temporary";
+ private ClassLoader ctxCL;
+ private ContextFactory ctxFactory = new ContextFactory();
+
+ @Override
+ public void open() throws HyracksDataException {
+ rd0 = inputRdFactory == null ? recordDescProvider.getInputRecordDescriptor(getActivityId(), 0)
+ : inputRdFactory.createRecordDescriptor(ctx);
+ frameDeserializer = new FrameDeserializer(ctx.getFrameSize(), rd0);
+ ctxCL = Thread.currentThread().getContextClassLoader();
+ Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader());
+ conf = confFactory.createConfiguration(ctx);
+
+ VertexOutputFormat outputFormat = BspUtils.createVertexOutputFormat(conf);
+ context = ctxFactory.createContext(conf, partition);
+ context.getConfiguration().setClassLoader(ctx.getJobletContext().getClassLoader());
+ try {
+ vertexWriter = outputFormat.createVertexWriter(context);
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void nextFrame(ByteBuffer frame) throws HyracksDataException {
+ frameDeserializer.reset(frame);
+ try {
+ while (!frameDeserializer.done()) {
+ Object[] tuple = frameDeserializer.deserializeRecord();
+ Vertex value = (Vertex) tuple[1];
+ vertexWriter.writeVertex(value);
+ }
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ Thread.currentThread().setContextClassLoader(ctxCL);
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ try {
+ vertexWriter.close(context);
+ moveFilesToFinalPath();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ private void moveFilesToFinalPath() throws HyracksDataException {
+ try {
+ JobContext job = ctxFactory.createJobContext(conf);
+ Path outputPath = FileOutputFormat.getOutputPath(job);
+ FileSystem dfs = FileSystem.get(conf);
+ Path filePath = new Path(outputPath, "part-" + new Integer(partition).toString());
+ FileStatus[] results = findPartitionPaths(outputPath, dfs);
+ if (results.length >= 1) {
+ /**
+ * for Hadoop-0.20.2
+ */
+ renameFile(dfs, filePath, results);
+ } else {
+ /**
+ * for Hadoop-0.23.1
+ */
+ int jobId = job.getJobID().getId();
+ outputPath = new Path(outputPath.toString() + File.separator + TEMP_DIR + File.separator
+ + jobId);
+ results = findPartitionPaths(outputPath, dfs);
+ renameFile(dfs, filePath, results);
+ }
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ } finally {
+ Thread.currentThread().setContextClassLoader(ctxCL);
+ }
+ }
+
+ private FileStatus[] findPartitionPaths(Path outputPath, FileSystem dfs) throws FileNotFoundException,
+ IOException {
+ FileStatus[] tempPaths = dfs.listStatus(outputPath, new PathFilter() {
+ @Override
+ public boolean accept(Path dir) {
+ return dir.getName().endsWith(TEMP_DIR) && dir.getName().indexOf(".crc") < 0;
+ }
+ });
+ Path tempDir = tempPaths[0].getPath();
+ FileStatus[] results = dfs.listStatus(tempDir, new PathFilter() {
+ @Override
+ public boolean accept(Path dir) {
+ return dir.getName().indexOf(context.getTaskAttemptID().toString()) >= 0
+ && dir.getName().indexOf(".crc") < 0;
+ }
+ });
+ return results;
+ }
+
+ private void renameFile(FileSystem dfs, Path filePath, FileStatus[] results) throws IOException,
+ HyracksDataException, FileNotFoundException {
+ Path srcDir = results[0].getPath();
+ if (!dfs.exists(srcDir))
+ throw new HyracksDataException("file " + srcDir.toString() + " does not exist!");
+
+ FileStatus[] srcFiles = dfs.listStatus(srcDir);
+ Path srcFile = srcFiles[0].getPath();
+ dfs.delete(filePath, true);
+ dfs.rename(srcFile, filePath);
+ }
+
+ };
+ }
+}
diff --git a/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java b/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
index bfe89ab..f3f7513 100644
--- a/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
+++ b/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java
@@ -147,11 +147,15 @@
return (RuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject();
}
- public synchronized void setVertexProperties(String jobId, long numVertices, long numEdges, int currentIteration) {
+ public synchronized void setVertexProperties(String jobId, long numVertices, long numEdges, long currentIteration) {
Boolean toMove = jobIdToMove.get(jobId);
if (toMove == null || toMove == true) {
if (jobIdToSuperStep.get(jobId) == null) {
- jobIdToSuperStep.put(jobId, 0L);
+ if (currentIteration <= 0) {
+ jobIdToSuperStep.put(jobId, 0L);
+ } else {
+ jobIdToSuperStep.put(jobId, currentIteration);
+ }
}
long superStep = jobIdToSuperStep.get(jobId);
@@ -175,6 +179,35 @@
System.gc();
}
+ public synchronized void recoverVertexProperties(String jobId, long numVertices, long numEdges,
+ long currentIteration) {
+ if (jobIdToSuperStep.get(jobId) == null) {
+ if (currentIteration <= 0) {
+ jobIdToSuperStep.put(jobId, 0L);
+ } else {
+ jobIdToSuperStep.put(jobId, currentIteration);
+ }
+ }
+
+ long superStep = jobIdToSuperStep.get(jobId);
+ List<FileReference> files = iterationToFiles.remove(superStep - 1);
+ if (files != null) {
+ for (FileReference fileRef : files)
+ fileRef.delete();
+ }
+
+ if (currentIteration > 0) {
+ Vertex.setSuperstep(currentIteration);
+ } else {
+ Vertex.setSuperstep(++superStep);
+ }
+ Vertex.setNumVertices(numVertices);
+ Vertex.setNumEdges(numEdges);
+ jobIdToSuperStep.put(jobId, superStep);
+ jobIdToMove.put(jobId, true);
+ LOGGER.info("recovered iteration " + Vertex.getSuperstep());
+ }
+
public synchronized void endSuperStep(String pregelixJobId) {
jobIdToMove.put(pregelixJobId, true);
LOGGER.info("end iteration " + Vertex.getSuperstep());
diff --git a/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java b/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java
index 1cf81ac..02097bf 100644
--- a/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java
+++ b/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java
@@ -75,13 +75,21 @@
context.endSuperStep(giraphJobId);
}
- public static void setProperties(String jobId, IHyracksTaskContext ctx, Configuration conf, int currentIteration) {
+ public static void setProperties(String jobId, IHyracksTaskContext ctx, Configuration conf, long currentIteration) {
INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
RuntimeContext context = (RuntimeContext) appContext.getApplicationObject();
context.setVertexProperties(jobId, conf.getLong(PregelixJob.NUM_VERTICE, -1),
conf.getLong(PregelixJob.NUM_EDGES, -1), currentIteration);
}
+ public static void recoverProperties(String jobId, IHyracksTaskContext ctx, Configuration conf,
+ long currentIteration) {
+ INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext();
+ RuntimeContext context = (RuntimeContext) appContext.getApplicationObject();
+ context.recoverVertexProperties(jobId, conf.getLong(PregelixJob.NUM_VERTICE, -1),
+ conf.getLong(PregelixJob.NUM_EDGES, -1), currentIteration);
+ }
+
public static void writeTerminationState(Configuration conf, String jobId, boolean terminate)
throws HyracksDataException {
try {
diff --git a/pregelix-dist/pom.xml b/pregelix-dist/pom.xml
index f0551a6..cec6efe 100644
--- a/pregelix-dist/pom.xml
+++ b/pregelix-dist/pom.xml
@@ -63,5 +63,10 @@
<artifactId>pregelix-example</artifactId>
<version>0.2.10-SNAPSHOT</version>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>pregelix-benchmark</artifactId>
+ <version>0.2.10-SNAPSHOT</version>
+ </dependency>
</dependencies>
</project>
diff --git a/pregelix-dist/src/main/resources/scripts/startcc.sh b/pregelix-dist/src/main/resources/scripts/startcc.sh
index 2a6cab2..d7a0ead 100644
--- a/pregelix-dist/src/main/resources/scripts/startcc.sh
+++ b/pregelix-dist/src/main/resources/scripts/startcc.sh
@@ -58,8 +58,8 @@
if [ -f "conf/topology.xml" ]; then
#Launch hyracks cc script with topology
-${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 -cluster-topology "conf/topology.xml" &> $CCLOGS_DIR/cc.log &
+${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -heartbeat-period 5000 -max-heartbeat-lapse-periods 4 -default-max-job-attempts 0 -job-history-size 0 -cluster-topology "conf/topology.xml" &> $CCLOGS_DIR/cc.log &
else
#Launch hyracks cc script without toplogy
-${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -max-heartbeat-lapse-periods 999999 -default-max-job-attempts 0 -job-history-size 0 &> $CCLOGS_DIR/cc.log &
+${PREGELIX_HOME}/bin/pregelixcc -client-net-ip-address $CCHOST -cluster-net-ip-address $CCHOST -client-net-port $CC_CLIENTPORT -cluster-net-port $CC_CLUSTERPORT -heartbeat-period 5000 -max-heartbeat-lapse-periods 4 -default-max-job-attempts 0 -job-history-size 0 &> $CCLOGS_DIR/cc.log &
fi
diff --git a/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/client/Client.java b/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/client/Client.java
index f99321a..393c8c9 100644
--- a/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/client/Client.java
+++ b/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/client/Client.java
@@ -45,7 +45,7 @@
public String ipAddress;
@Option(name = "-port", usage = "port of cluster controller", required = false)
- public int port;
+ public int port = 3099;
@Option(name = "-plan", usage = "query plan choice", required = false)
public Plan planChoice = Plan.OUTER_JOIN;
@@ -67,6 +67,9 @@
@Option(name = "-runtime-profiling", usage = "whether to do runtime profifling", required = false)
public String profiling = "false";
+
+ @Option(name = "-ckp-interval", usage = "checkpointing interval -- for fault-tolerance", required = false)
+ public int ckpInterval = -1;
}
public static void run(String[] args, PregelixJob job) throws Exception {
@@ -125,6 +128,7 @@
job.getConfiguration().setLong(ReachabilityVertex.DEST_ID, options.destId);
if (options.numIteration > 0)
job.getConfiguration().setLong(PageRankVertex.ITERATIONS, options.numIteration);
+ job.setCheckpointingInterval(options.ckpInterval);
}
}
diff --git a/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/data/VLongNormalizedKeyComputer.java b/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/data/VLongNormalizedKeyComputer.java
index 90065c2..44d23a4 100644
--- a/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/data/VLongNormalizedKeyComputer.java
+++ b/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/data/VLongNormalizedKeyComputer.java
@@ -14,13 +14,8 @@
*/
package edu.uci.ics.pregelix.example.data;
-import java.io.DataInput;
-import java.io.DataInputStream;
-
-import org.apache.hadoop.io.WritableUtils;
-
import edu.uci.ics.pregelix.api.graph.NormalizedKeyComputer;
-import edu.uci.ics.pregelix.api.util.ResetableByteArrayInputStream;
+import edu.uci.ics.pregelix.example.utils.SerDeUtils;
/**
* @author yingyib
@@ -31,14 +26,10 @@
private static final int NON_NEGATIVE_INT_MASK = (2 << 30);
private static final int NEGATIVE_LONG_MASK = (0 << 30);
- private ResetableByteArrayInputStream bis = new ResetableByteArrayInputStream();
- private DataInput dis = new DataInputStream(bis);
-
@Override
public int getNormalizedKey(byte[] bytes, int start, int length) {
try {
- bis.setByteArray(bytes, start);
- long value = WritableUtils.readVLong(dis);
+ long value = SerDeUtils.readVLong(bytes, start, length);
int highValue = (int) (value >> 32);
if (highValue > 0) {
/**
diff --git a/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/io/VLongWritable.java b/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/io/VLongWritable.java
index 1c5f629..ffbbff4 100644
--- a/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/io/VLongWritable.java
+++ b/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/io/VLongWritable.java
@@ -16,14 +16,12 @@
package edu.uci.ics.pregelix.example.io;
import java.io.DataInput;
-import java.io.DataInputStream;
import org.apache.hadoop.io.WritableComparable;
import org.apache.hadoop.io.WritableComparator;
-import org.apache.hadoop.io.WritableUtils;
import edu.uci.ics.pregelix.api.io.WritableSizable;
-import edu.uci.ics.pregelix.api.util.ResetableByteArrayInputStream;
+import edu.uci.ics.pregelix.example.utils.SerDeUtils;
/**
* A WritableComparable for longs in a variable-length format. Such values take
@@ -65,8 +63,6 @@
/** A Comparator optimized for LongWritable. */
public static class Comparator extends WritableComparator {
- private ResetableByteArrayInputStream bis = new ResetableByteArrayInputStream();
- private DataInput dis = new DataInputStream(bis);
public Comparator() {
super(VLongWritable.class);
@@ -74,10 +70,8 @@
public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
try {
- bis.setByteArray(b1, s1);
- long thisValue = WritableUtils.readVLong(dis);
- bis.setByteArray(b2, s2);
- long thatValue = WritableUtils.readVLong(dis);
+ long thisValue = SerDeUtils.readVLong(b1, s1, l1);
+ long thatValue = SerDeUtils.readVLong(b2, s2, l2);
return (thisValue < thatValue ? -1 : (thisValue == thatValue ? 0 : 1));
} catch (Exception e) {
throw new IllegalStateException(e);
diff --git a/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/SerDeUtils.java b/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/SerDeUtils.java
new file mode 100644
index 0000000..2800187
--- /dev/null
+++ b/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/utils/SerDeUtils.java
@@ -0,0 +1,56 @@
+package edu.uci.ics.pregelix.example.utils;
+
+import java.io.IOException;
+
+public class SerDeUtils {
+
+ /**
+ * Reads a zero-compressed encoded long from input stream and returns it.
+ *
+ * @param stream
+ * Binary input stream
+ * @throws java.io.IOException
+ * @return deserialized long from stream.
+ */
+ public static long readVLong(byte[] data, int start, int length) throws IOException {
+ byte firstByte = data[start];
+ int len = decodeVIntSize(firstByte);
+ if (len == 1) {
+ return firstByte;
+ }
+ long i = 0;
+ for (int idx = 0; idx < len - 1; idx++) {
+ i = i << 8;
+ i = i | (data[++start] & 0xFF);
+ }
+ return (isNegativeVInt(firstByte) ? (i ^ -1L) : i);
+ }
+
+ /**
+ * Parse the first byte of a vint/vlong to determine the number of bytes
+ *
+ * @param value
+ * the first byte of the vint/vlong
+ * @return the total number of bytes (1 to 9)
+ */
+ public static int decodeVIntSize(byte value) {
+ if (value >= -112) {
+ return 1;
+ } else if (value < -120) {
+ return -119 - value;
+ }
+ return -111 - value;
+ }
+
+ /**
+ * Given the first byte of a vint/vlong, determine the sign
+ *
+ * @param value
+ * the first byte
+ * @return is the value negative
+ */
+ public static boolean isNegativeVInt(byte value) {
+ return value < -120 || (value >= -112 && value < 0);
+ }
+
+}
diff --git a/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryConnectedComponentsTest.java b/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryConnectedComponentsTest.java
new file mode 100644
index 0000000..efc7bcc
--- /dev/null
+++ b/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryConnectedComponentsTest.java
@@ -0,0 +1,91 @@
+/*
+ * 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.io.File;
+
+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.graph.Vertex;
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook;
+import edu.uci.ics.pregelix.api.util.DefaultVertexPartitioner;
+import edu.uci.ics.pregelix.core.driver.Driver;
+import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
+import edu.uci.ics.pregelix.example.ConnectedComponentsVertex.SimpleConnectedComponentsVertexOutputFormat;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
+import edu.uci.ics.pregelix.example.inputformat.TextConnectedComponentsInputFormat;
+import edu.uci.ics.pregelix.example.util.TestCluster;
+import edu.uci.ics.pregelix.example.util.TestUtils;
+
+/**
+ * @author yingyib
+ */
+public class FailureRecoveryConnectedComponentsTest {
+ private static String INPUTPATH = "data/webmapcomplex";
+ private static String OUTPUTPAH = "actual/result";
+ private static String EXPECTEDPATH = "src/test/resources/expected/ConnectedComponentsRealComplex2";
+
+ @Test
+ public void test() throws Exception {
+ TestCluster testCluster = new TestCluster();
+ try {
+ PregelixJob job = new PregelixJob(ConnectedComponentsVertex.class.getName());
+ job.setVertexClass(ConnectedComponentsVertex.class);
+ job.setVertexClass(ConnectedComponentsVertex.class);
+ job.setVertexInputFormatClass(TextConnectedComponentsInputFormat.class);
+ job.setVertexOutputFormatClass(SimpleConnectedComponentsVertexOutputFormat.class);
+ job.setMessageCombinerClass(ConnectedComponentsVertex.SimpleMinCombiner.class);
+ job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+ job.setVertexPartitionerClass(DefaultVertexPartitioner.class);
+ job.setDynamicVertexValueSize(true);
+ FileInputFormat.setInputPaths(job, INPUTPATH);
+ FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH));
+ job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23);
+ job.setCheckpointHook(ConservativeCheckpointHook.class);
+
+ testCluster.setUp();
+ Driver driver = new Driver(PageRankVertex.class);
+ Thread thread = new Thread(new Runnable() {
+
+ @Override
+ public void run() {
+ try {
+ synchronized (this) {
+ while (Vertex.getSuperstep() <= 5) {
+ this.wait(200);
+ }
+ PregelixHyracksIntegrationUtil.shutdownNC1();
+ }
+ } catch (Exception e) {
+ throw new IllegalStateException(e);
+ }
+ }
+ });
+ thread.start();
+ driver.runJob(job, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT);
+
+ TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH));
+ } catch (Exception e) {
+ PregelixHyracksIntegrationUtil.shutdownNC2();
+ testCluster.cleanupHDFS();
+ throw e;
+ }
+ }
+
+}
diff --git a/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryInnerJoinTest.java b/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryInnerJoinTest.java
new file mode 100644
index 0000000..ff1e29f
--- /dev/null
+++ b/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryInnerJoinTest.java
@@ -0,0 +1,90 @@
+/*
+ * 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.io.File;
+
+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.graph.Vertex;
+import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook;
+import edu.uci.ics.pregelix.core.base.IDriver.Plan;
+import edu.uci.ics.pregelix.core.driver.Driver;
+import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
+import edu.uci.ics.pregelix.example.PageRankVertex.SimplePageRankVertexOutputFormat;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
+import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
+import edu.uci.ics.pregelix.example.util.TestCluster;
+import edu.uci.ics.pregelix.example.util.TestUtils;
+
+/**
+ * @author yingyib
+ */
+public class FailureRecoveryInnerJoinTest {
+ private static String INPUTPATH = "data/webmap";
+ private static String OUTPUTPAH = "actual/result";
+ private static String EXPECTEDPATH = "src/test/resources/expected/PageRankReal2";
+
+ @Test
+ public void test() throws Exception {
+ TestCluster testCluster = new TestCluster();
+
+ try {
+ PregelixJob job = new PregelixJob(PageRankVertex.class.getName());
+ job.setVertexClass(PageRankVertex.class);
+ job.setVertexInputFormatClass(TextPageRankInputFormat.class);
+ job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
+ job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
+ job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+ FileInputFormat.setInputPaths(job, INPUTPATH);
+ FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH));
+ job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
+ job.setCheckpointHook(ConservativeCheckpointHook.class);
+
+ testCluster.setUp();
+ Driver driver = new Driver(PageRankVertex.class);
+ Thread thread = new Thread(new Runnable() {
+
+ @Override
+ public void run() {
+ try {
+ synchronized (this) {
+ while (Vertex.getSuperstep() <= 5) {
+ this.wait(200);
+ }
+ PregelixHyracksIntegrationUtil.shutdownNC1();
+ }
+ } catch (Exception e) {
+ throw new IllegalStateException(e);
+ }
+ }
+ });
+ thread.start();
+ driver.runJob(job, Plan.INNER_JOIN, "127.0.0.1",
+ PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT, false);
+
+ TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH));
+ } catch (Exception e) {
+ PregelixHyracksIntegrationUtil.shutdownNC2();
+ testCluster.cleanupHDFS();
+ throw e;
+ }
+ }
+
+}
diff --git a/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java b/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java
index dac087f..3fdaf15 100644
--- a/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java
+++ b/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java
@@ -21,6 +21,7 @@
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
import org.junit.Test;
+import edu.uci.ics.pregelix.api.graph.Vertex;
import edu.uci.ics.pregelix.api.job.PregelixJob;
import edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook;
import edu.uci.ics.pregelix.core.driver.Driver;
@@ -37,7 +38,7 @@
public class FailureRecoveryTest {
private static String INPUTPATH = "data/webmap";
private static String OUTPUTPAH = "actual/result";
- private static String EXPECTEDPATH = "src/test/resources/expected/PageRankReal";
+ private static String EXPECTEDPATH = "src/test/resources/expected/PageRankReal2";
@Test
public void test() throws Exception {
@@ -57,29 +58,30 @@
testCluster.setUp();
Driver driver = new Driver(PageRankVertex.class);
- // Thread thread = new Thread(new Runnable() {
- //
- // @Override
- // public void run() {
- // try {
- // synchronized (this) {
- // this.wait(10000);
- // PregelixHyracksIntegrationUtil.showDownNC1();
- // }
- // } catch (Exception e) {
- // throw new IllegalStateException(e);
- // }
- // }
- //
- // });
- //thread.start();
+ Thread thread = new Thread(new Runnable() {
+
+ @Override
+ public void run() {
+ try {
+ synchronized (this) {
+ while (Vertex.getSuperstep() <= 5) {
+ this.wait(200);
+ }
+ PregelixHyracksIntegrationUtil.shutdownNC1();
+ }
+ } catch (Exception e) {
+ throw new IllegalStateException(e);
+ }
+ }
+ });
+ thread.start();
driver.runJob(job, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT);
TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH));
} catch (Exception e) {
+ PregelixHyracksIntegrationUtil.shutdownNC2();
+ testCluster.cleanupHDFS();
throw e;
- } finally {
- testCluster.tearDown();
}
}
diff --git a/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryWithoutCheckpointTest.java b/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryWithoutCheckpointTest.java
new file mode 100644
index 0000000..e006ccd
--- /dev/null
+++ b/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryWithoutCheckpointTest.java
@@ -0,0 +1,86 @@
+/*
+ * 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.io.File;
+
+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.graph.Vertex;
+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.PageRankVertex.SimplePageRankVertexOutputFormat;
+import edu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer;
+import edu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat;
+import edu.uci.ics.pregelix.example.util.TestCluster;
+import edu.uci.ics.pregelix.example.util.TestUtils;
+
+/**
+ * @author yingyib
+ */
+public class FailureRecoveryWithoutCheckpointTest {
+ private static String INPUTPATH = "data/webmap";
+ private static String OUTPUTPAH = "actual/result";
+ private static String EXPECTEDPATH = "src/test/resources/expected/PageRankReal2";
+
+ @Test
+ public void test() throws Exception {
+ TestCluster testCluster = new TestCluster();
+
+ try {
+ PregelixJob job = new PregelixJob(PageRankVertex.class.getName());
+ job.setVertexClass(PageRankVertex.class);
+ job.setVertexInputFormatClass(TextPageRankInputFormat.class);
+ job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class);
+ job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class);
+ job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
+ FileInputFormat.setInputPaths(job, INPUTPATH);
+ FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH));
+ job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
+
+ testCluster.setUp();
+ Driver driver = new Driver(PageRankVertex.class);
+ Thread thread = new Thread(new Runnable() {
+
+ @Override
+ public void run() {
+ try {
+ synchronized (this) {
+ while (Vertex.getSuperstep() <= 5) {
+ this.wait(200);
+ }
+ PregelixHyracksIntegrationUtil.shutdownNC1();
+ }
+ } catch (Exception e) {
+ throw new IllegalStateException(e);
+ }
+ }
+ });
+ thread.start();
+ driver.runJob(job, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT);
+
+ TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH));
+ } catch (Exception e) {
+ PregelixHyracksIntegrationUtil.shutdownNC2();
+ testCluster.cleanupHDFS();
+ throw e;
+ }
+ }
+
+}
diff --git a/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/FailureVertex.java b/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureVertex.java
similarity index 100%
rename from pregelix-example/src/main/java/edu/uci/ics/pregelix/example/FailureVertex.java
rename to pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureVertex.java
diff --git a/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/JobConcatenationTest.java b/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/JobConcatenationTest.java
index 5a485ba..dc7a28d 100644
--- a/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/JobConcatenationTest.java
+++ b/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/JobConcatenationTest.java
@@ -24,6 +24,7 @@
import org.junit.Test;
import edu.uci.ics.pregelix.api.job.PregelixJob;
+import edu.uci.ics.pregelix.api.util.ConservativeCheckpointHook;
import edu.uci.ics.pregelix.core.driver.Driver;
import edu.uci.ics.pregelix.core.util.PregelixHyracksIntegrationUtil;
import edu.uci.ics.pregelix.example.PageRankVertex.SimplePageRankVertexOutputFormat;
@@ -55,7 +56,7 @@
job1.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
FileInputFormat.setInputPaths(job1, INPUTPATH);
job1.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
- //job1.setCheckpointHook(ConservativeCheckpointHook.class);
+ job1.setCheckpointHook(ConservativeCheckpointHook.class);
PregelixJob job2 = new PregelixJob(PageRankVertex.class.getName());
job2.setVertexClass(PageRankVertex.class);
@@ -65,7 +66,7 @@
job2.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class);
FileOutputFormat.setOutputPath(job2, new Path(OUTPUTPAH));
job2.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20);
- //job2.setCheckpointHook(ConservativeCheckpointHook.class);
+ job2.setCheckpointHook(ConservativeCheckpointHook.class);
jobs.add(job1);
jobs.add(job2);
diff --git a/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestCluster.java b/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestCluster.java
index 40ea690..660d9eb 100644
--- a/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestCluster.java
+++ b/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/util/TestCluster.java
@@ -126,7 +126,7 @@
/**
* cleanup hdfs cluster
*/
- private void cleanupHDFS() throws Exception {
+ public void cleanupHDFS() throws Exception {
dfsCluster.shutdown();
}
diff --git a/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-0 b/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-0
new file mode 100755
index 0000000..2c975de
--- /dev/null
+++ b/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-0
@@ -0,0 +1,10 @@
+0 0
+2 0
+4 0
+6 0
+8 0
+10 0
+12 0
+14 0
+16 0
+18 0
diff --git a/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-1 b/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-1
new file mode 100755
index 0000000..6976bc1
--- /dev/null
+++ b/pregelix-example/src/test/resources/expected/ConnectedComponentsRealComplex2/part-1
@@ -0,0 +1,13 @@
+1 0
+3 0
+5 0
+7 0
+9 0
+11 0
+13 0
+15 0
+17 0
+19 0
+21 21
+25 25
+27 27
diff --git a/pregelix-example/src/test/resources/expected/PageRankReal2/part-0 b/pregelix-example/src/test/resources/expected/PageRankReal2/part-0
new file mode 100755
index 0000000..d135b86
--- /dev/null
+++ b/pregelix-example/src/test/resources/expected/PageRankReal2/part-0
@@ -0,0 +1,10 @@
+0 0.008290140026154316
+2 0.14646839195826472
+4 0.03976979906329426
+6 0.015736276824953852
+8 0.010628239626209894
+10 0.008290140026154316
+12 0.14646839195826472
+14 0.03976979906329426
+16 0.015736276824953852
+18 0.010628239626209894
diff --git a/pregelix-example/src/test/resources/expected/PageRankReal2/part-1 b/pregelix-example/src/test/resources/expected/PageRankReal2/part-1
new file mode 100755
index 0000000..d3badee
--- /dev/null
+++ b/pregelix-example/src/test/resources/expected/PageRankReal2/part-1
@@ -0,0 +1,10 @@
+1 0.15351528192471647
+3 0.08125113985998214
+5 0.0225041581462058
+7 0.012542224114863661
+9 0.009294348455354817
+11 0.15351528192471647
+13 0.08125113985998214
+15 0.0225041581462058
+17 0.012542224114863661
+19 0.009294348455354817
diff --git a/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RecoveryRuntimeHookFactory.java b/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RecoveryRuntimeHookFactory.java
index 35e7cd8..4720272 100644
--- a/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RecoveryRuntimeHookFactory.java
+++ b/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RecoveryRuntimeHookFactory.java
@@ -30,11 +30,11 @@
*/
public class RecoveryRuntimeHookFactory implements IRuntimeHookFactory {
private static final long serialVersionUID = 1L;
- private final int currentSuperStep;
+ private final long currentSuperStep;
private String jobId;
private IConfigurationFactory confFactory;
- public RecoveryRuntimeHookFactory(String jobId, int currentSuperStep, IConfigurationFactory confFactory) {
+ public RecoveryRuntimeHookFactory(String jobId, long currentSuperStep, IConfigurationFactory confFactory) {
this.currentSuperStep = currentSuperStep;
this.jobId = jobId;
this.confFactory = confFactory;
@@ -48,7 +48,7 @@
public void configure(IHyracksTaskContext ctx) throws HyracksDataException {
IterationUtils.endSuperStep(jobId, ctx);
Configuration conf = confFactory.createConfiguration(ctx);
- IterationUtils.setProperties(jobId, ctx, conf, currentSuperStep);
+ IterationUtils.recoverProperties(jobId, ctx, conf, currentSuperStep);
}
};