diff --git a/.gitignore b/.gitignore index 18a13eb2d..11c2d8ee8 100644 --- a/.gitignore +++ b/.gitignore @@ -1,3 +1,4 @@ +bin target .classpath .settings diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java index c5326ce5d..956c74f9b 100644 --- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java +++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java @@ -30,6 +30,8 @@ import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator; import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema; import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator; +import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty; +import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator; import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector; import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements; import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector; @@ -60,7 +62,6 @@ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext @Override public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op, IPhysicalPropertiesVector reqdByParent) { - return emptyUnaryRequirements(); } diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceOrderByAfterSubplan.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceOrderByAfterSubplan.java index a684bc97b..02e090d8e 100644 --- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceOrderByAfterSubplan.java +++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceOrderByAfterSubplan.java @@ -104,6 +104,10 @@ public boolean rewritePost(Mutable opRef, IOptimizationContext foundTarget = false; break; } + if(child.getOperatorTag() == LogicalOperatorTag.GROUP){ + foundTarget = false; + break; + } if (orderSensitiveOps.contains(child.getOperatorTag())) { orderSensitive = true; } diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java index 98606f129..810defc5e 100644 --- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java +++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java @@ -270,6 +270,8 @@ private boolean physOptimizeOp(Mutable opRef, IPhysicalPropert // Now, transfer annotations from the original sort op. to this one. AbstractLogicalOperator transferTo = nextOp; if (transferTo.getOperatorTag() == LogicalOperatorTag.EXCHANGE) { + // + // remove duplicate exchange operator transferTo = (AbstractLogicalOperator) transferTo.getInputs().get(0).getValue(); } transferTo.getAnnotations().putAll(op.getAnnotations()); @@ -345,6 +347,13 @@ private List getOrderColumnsFromGroupingProperties(List opRef, IOptimizationContext throws AlgebricksException { AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue(); if (op1.getPhysicalOperator() == null - || op1.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.HASH_PARTITION_EXCHANGE) { + || (op1.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.HASH_PARTITION_EXCHANGE && op1 + .getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.HASH_PARTITION_MERGE_EXCHANGE)) { return false; } AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getValue(); @@ -51,6 +52,12 @@ public boolean rewritePost(Mutable opRef, IOptimizationContext || op2.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.SORT_MERGE_EXCHANGE) { return false; } + if (op1.getPhysicalOperator().getOperatorTag() == PhysicalOperatorTag.HASH_PARTITION_MERGE_EXCHANGE) { + // if it is a hash_partition_merge_exchange, the sort_merge_exchange can be simply removed + op1.getInputs().get(0).setValue(op2.getInputs().get(0).getValue()); + op1.computeDeliveredPhysicalProperties(context); + return true; + } HashPartitionExchangePOperator hpe = (HashPartitionExchangePOperator) op1.getPhysicalOperator(); SortMergeExchangePOperator sme = (SortMergeExchangePOperator) op2.getPhysicalOperator(); List ocList = new ArrayList(); diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java index 2a0f6c389..756e0f33f 100644 --- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java +++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PullSelectOutOfEqJoin.java @@ -48,8 +48,7 @@ public boolean rewritePost(Mutable opRef, IOptimizationContext throws AlgebricksException { AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue(); - if (op.getOperatorTag() != LogicalOperatorTag.INNERJOIN - && op.getOperatorTag() != LogicalOperatorTag.LEFTOUTERJOIN) { + if (op.getOperatorTag() != LogicalOperatorTag.INNERJOIN) { return false; } AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) op; diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java index dc9c80566..8973c7540 100644 --- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java +++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/aggreg/NestedPlansRunningAggregatorFactory.java @@ -131,7 +131,14 @@ public boolean outputPartialResult(ArrayTupleBuilder tupleBuilder, IFrameTupleAc @Override public void close() { - + for (int i = 0; i < pipelines.length; ++i) { + try { + outputWriter.setInputIdx(i); + pipelines[i].close(); + } catch (HyracksDataException e) { + throw new IllegalStateException(e); + } + } } }; } @@ -232,7 +239,10 @@ public void nextFrame(ByteBuffer buffer) throws HyracksDataException { @Override public void close() throws HyracksDataException { - // clearFrame(); + if(outputAppender.getTupleCount() > 0){ + FrameUtils.flushFrame(outputFrame, outputWriter); + outputAppender.reset(outputFrame, true); + } } public void setInputIdx(int inputIdx) { diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java index cfc2bb6b5..600d641cc 100644 --- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java +++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/RunningAggregateRuntimeFactory.java @@ -90,7 +90,7 @@ public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(f @Override public void open() throws HyracksDataException { - if(!first){ + if (!first) { FrameUtils.flushFrame(frame, writer); appender.reset(frame, true); } diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java index 88dc19fe0..415e718ba 100644 --- a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java +++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/UnnestRuntimeFactory.java @@ -86,7 +86,6 @@ public AbstractOneInputOneOutputOneFramePushRuntime createOneOutputPushRuntime(f private IUnnestingEvaluator agg; private ArrayTupleBuilder tupleBuilder; - private int tupleCount; private IScalarEvaluator offsetEval = posOffsetEvalFactory.createScalarEvaluator(ctx); @Override @@ -98,7 +97,6 @@ public void open() throws HyracksDataException { throw new HyracksDataException(ae); } tupleBuilder = new ArrayTupleBuilder(projectionList.length); - tupleCount = 1; writer.open(); } @@ -120,6 +118,10 @@ public void nextFrame(ByteBuffer buffer) throws HyracksDataException { try { agg.init(tRef); + // assume that when unnesting the tuple, each step() call for each element + // in the tuple will increase the positionIndex, and the positionIndex will + // be reset when a new tuple is to be processed. + int positionIndex = 1; boolean goon = true; do { tupleBuilder.reset(); @@ -146,7 +148,7 @@ public void nextFrame(ByteBuffer buffer) throws HyracksDataException { if (hasPositionalVariable) { // Write the positional variable as an INT32 tupleBuilder.getDataOutput().writeByte(3); - tupleBuilder.getDataOutput().writeInt(offset + tupleCount++); + tupleBuilder.getDataOutput().writeInt(offset + positionIndex++); tupleBuilder.addFieldEndOffset(); } appendToFrameFromTupleBuilder(tupleBuilder); diff --git a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/DeBruijnGraphCleanVertex.java b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/DeBruijnGraphCleanVertex.java index 00d596e9d..f9cfdbf3f 100644 --- a/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/DeBruijnGraphCleanVertex.java +++ b/genomix/genomix-pregelix/src/main/java/edu/uci/ics/genomix/pregelix/operator/DeBruijnGraphCleanVertex.java @@ -150,7 +150,7 @@ public void incrementCounter(byte counterName) { public static HashMapWritable readStatisticsCounterResult(Configuration conf) { try { VertexValueWritable value = (VertexValueWritable) IterationUtils.readGlobalAggregateValue(conf, - BspUtils.getJobId(conf)); + BspUtils.getJobId(conf), StatisticsAggregator.class.getName()); return value.getCounters(); } catch (IOException e) { throw new IllegalStateException(e); @@ -169,7 +169,7 @@ public static PregelixJob getConfiguredJob( else job = new PregelixJob(conf, vertexClass.getSimpleName()); job.setVertexClass(vertexClass); - job.setGlobalAggregatorClass(StatisticsAggregator.class); + job.addGlobalAggregatorClass(StatisticsAggregator.class); job.setVertexInputFormatClass(NodeToVertexInputFormat.class); job.setVertexOutputFormatClass(VertexToNodeOutputFormat.class); job.setOutputKeyClass(VKmer.class); diff --git a/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/jobgen/JobGenerator.java b/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/jobgen/JobGenerator.java index 36e7f996d..8d3b9310d 100644 --- a/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/jobgen/JobGenerator.java +++ b/genomix/genomix-pregelix/src/test/java/edu/uci/ics/genomix/pregelix/jobgen/JobGenerator.java @@ -39,7 +39,7 @@ public class JobGenerator { public static String outputBase = "src/test/resources/jobs/"; private static void configureJob(PregelixJob job) { - job.setGlobalAggregatorClass(StatisticsAggregator.class); + job.addGlobalAggregatorClass(StatisticsAggregator.class); job.setVertexInputFormatClass(NodeToVertexInputFormat.class); job.setVertexOutputFormatClass(VertexToNodeOutputFormat.class); job.setDynamicVertexValueSize(true); diff --git a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q21_suppliers_who_kept_orders_waiting.plan b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q21_suppliers_who_kept_orders_waiting.plan index a22bf5313..e4f2cd6eb 100644 --- a/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q21_suppliers_who_kept_orders_waiting.plan +++ b/hivesterix/hivesterix-dist/src/test/resources/optimizerts/results/q21_suppliers_who_kept_orders_waiting.plan @@ -14,7 +14,7 @@ write [%0->$$21, %0->$$24, %0->$$23] } -- PRE_CLUSTERED_GROUP_BY[$$17] |PARTITIONED| exchange - -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$17(ASC)] HASH:[$$17] |PARTITIONED| + -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$17(ASC), $$18(ASC)] HASH:[$$17] |PARTITIONED| group by ([$$17 := %0->$$1; $$18 := %0->$$3]) decor ([]) { aggregate [$$20] <- [function-call: hive:max(PARTIAL1), Args:[%0->$$3]] -- AGGREGATE |LOCAL| @@ -50,7 +50,7 @@ write [%0->$$21, %0->$$24, %0->$$23] } -- PRE_CLUSTERED_GROUP_BY[$$17] |PARTITIONED| exchange - -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$17(ASC)] HASH:[$$17] |PARTITIONED| + -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$17(ASC), $$18(ASC)] HASH:[$$17] |PARTITIONED| group by ([$$17 := %0->$$1; $$18 := %0->$$3]) decor ([]) { aggregate [$$20] <- [function-call: hive:max(PARTIAL1), Args:[%0->$$3]] -- AGGREGATE |LOCAL| diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSerializerDeserializerContainer.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSerializerDeserializerContainer.java index 0ce23461a..57c9133eb 100644 --- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSerializerDeserializerContainer.java +++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSerializerDeserializerContainer.java @@ -25,7 +25,7 @@ public interface IJobSerializerDeserializerContainer { * @param deploymentId * @return */ - public IJobSerializerDeserializer getJobSerializerDeerializer(DeploymentId deploymentId); + public IJobSerializerDeserializer getJobSerializerDeserializer(DeploymentId deploymentId); /** * Add a deployment with the job serializer deserializer diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSerializerDeserializerContainer.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSerializerDeserializerContainer.java index 35a1e8be2..f2d8fff48 100644 --- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSerializerDeserializerContainer.java +++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSerializerDeserializerContainer.java @@ -15,18 +15,18 @@ package edu.uci.ics.hyracks.api.job; -import java.util.HashMap; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import edu.uci.ics.hyracks.api.deployment.DeploymentId; public class JobSerializerDeserializerContainer implements IJobSerializerDeserializerContainer { private IJobSerializerDeserializer defaultJobSerDe = new JobSerializerDeserializer(); - private Map jobSerializerDeserializerMap = new HashMap(); + private Map jobSerializerDeserializerMap = new ConcurrentHashMap(); @Override - public synchronized IJobSerializerDeserializer getJobSerializerDeerializer(DeploymentId deploymentId) { + public synchronized IJobSerializerDeserializer getJobSerializerDeserializer(DeploymentId deploymentId) { if (deploymentId == null) { return defaultJobSerDe; } @@ -44,4 +44,9 @@ public synchronized void removeJobSerializerDeserializer(DeploymentId deployment jobSerializerDeserializerMap.remove(deploymentId); } + @Override + public String toString() { + return jobSerializerDeserializerMap.toString(); + } + } diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/lifecycle/LifeCycleComponentManager.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/lifecycle/LifeCycleComponentManager.java index ec276535c..ad708e9bd 100644 --- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/lifecycle/LifeCycleComponentManager.java +++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/lifecycle/LifeCycleComponentManager.java @@ -35,6 +35,7 @@ public static final class Config { private final List components; private boolean stopInitiated; + private boolean stopped; private String dumpPath; private boolean configured; @@ -42,6 +43,7 @@ private LifeCycleComponentManager() { components = new ArrayList(); stopInitiated = false; configured = false; + stopped = false; } @Override @@ -76,6 +78,12 @@ public synchronized void stopAll(boolean dumpState) throws IOException { if (LOGGER.isLoggable(Level.INFO)) { LOGGER.severe("Attempting to stop " + this); } + if (stopped) { + if (LOGGER.isLoggable(Level.INFO)) { + LOGGER.severe("Lifecycle management was already stopped"); + } + return; + } if (stopInitiated) { if (LOGGER.isLoggable(Level.INFO)) { LOGGER.severe("Stop already in progress"); @@ -124,7 +132,7 @@ public synchronized void stopAll(boolean dumpState) throws IOException { } } stopInitiated = false; - + stopped = true; } @Override @@ -142,4 +150,8 @@ public void configure(Map configuration) { configured = true; } + public boolean stoppedAll() { + return stopped; + } + } diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java index 6785d6f69..85ce1048e 100644 --- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java +++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java @@ -219,8 +219,8 @@ public void notifyHeartbeat(HeartbeatData hbData) { ipcMessageBytesSent[rrdPtr] = hbData.ipcMessageBytesSent; ipcMessagesReceived[rrdPtr] = hbData.ipcMessagesReceived; ipcMessageBytesReceived[rrdPtr] = hbData.ipcMessageBytesReceived; + rrdPtr = (rrdPtr + 1) % RRD_SIZE; } - rrdPtr = (rrdPtr + 1) % RRD_SIZE; } public int incrementLastHeartbeatDuration() { diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java index 2ca7ccf84..828f2fb98 100644 --- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java +++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/application/ApplicationContext.java @@ -31,7 +31,9 @@ public abstract class ApplicationContext implements IApplicationContext { protected IJobSerializerDeserializerContainer jobSerDeContainer = new JobSerializerDeserializerContainer(); protected ThreadFactory threadFactory = new ThreadFactory() { public Thread newThread(Runnable r) { - return new Thread(r); + Thread t = new Thread(r); + t.setDaemon(true); + return t; } }; diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/ClassLoaderJobSerializerDeserializer.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/ClassLoaderJobSerializerDeserializer.java index 3a35c2537..9be381805 100644 --- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/ClassLoaderJobSerializerDeserializer.java +++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/ClassLoaderJobSerializerDeserializer.java @@ -31,7 +31,6 @@ * This is the IJobSerializerDeserializer implementation for jobs with dynamic deployed jars. * * @author yingyib - * */ public class ClassLoaderJobSerializerDeserializer implements IJobSerializerDeserializer { @@ -99,4 +98,9 @@ public Class loadClass(String className) throws HyracksException { public ClassLoader getClassLoader() throws HyracksException { return classLoader; } + + @Override + public String toString() { + return classLoader.toString(); + } } diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentUtils.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentUtils.java index 0677e2ea4..0724a01a1 100644 --- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentUtils.java +++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/deployment/DeploymentUtils.java @@ -89,7 +89,7 @@ public static void undeploy(DeploymentId deploymentId, IJobSerializerDeserialize */ public static void deploy(DeploymentId deploymentId, List urls, IJobSerializerDeserializerContainer container, ServerContext ctx, boolean isNC) throws HyracksException { - IJobSerializerDeserializer jobSerDe = container.getJobSerializerDeerializer(deploymentId); + IJobSerializerDeserializer jobSerDe = container.getJobSerializerDeserializer(deploymentId); if (jobSerDe == null) { jobSerDe = new ClassLoaderJobSerializerDeserializer(); container.addJobSerializerDeserializer(deploymentId, jobSerDe); @@ -116,7 +116,7 @@ public static Object deserialize(byte[] bytes, DeploymentId deploymentId, IAppli try { IJobSerializerDeserializerContainer jobSerDeContainer = appCtx.getJobSerializerDeserializerContainer(); IJobSerializerDeserializer jobSerDe = deploymentId == null ? null : jobSerDeContainer - .getJobSerializerDeerializer(deploymentId); + .getJobSerializerDeserializer(deploymentId); Object obj = jobSerDe == null ? JavaSerializationUtils.deserialize(bytes) : jobSerDe.deserialize(bytes); return obj; } catch (Exception e) { @@ -138,7 +138,7 @@ public static Class loadClass(String className, DeploymentId deploymentId, IA try { IJobSerializerDeserializerContainer jobSerDeContainer = appCtx.getJobSerializerDeserializerContainer(); IJobSerializerDeserializer jobSerDe = deploymentId == null ? null : jobSerDeContainer - .getJobSerializerDeerializer(deploymentId); + .getJobSerializerDeserializer(deploymentId); Class cl = jobSerDe == null ? JavaSerializationUtils.loadClass(className) : jobSerDe .loadClass(className); return cl; @@ -160,7 +160,7 @@ public static ClassLoader getClassLoader(DeploymentId deploymentId, IApplication try { IJobSerializerDeserializerContainer jobSerDeContainer = appCtx.getJobSerializerDeserializerContainer(); IJobSerializerDeserializer jobSerDe = deploymentId == null ? null : jobSerDeContainer - .getJobSerializerDeerializer(deploymentId); + .getJobSerializerDeserializer(deploymentId); ClassLoader cl = jobSerDe == null ? DeploymentUtils.class.getClassLoader() : jobSerDe.getClassLoader(); return cl; } catch (Exception e) { diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java index 6049a3bfa..245d9886b 100644 --- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java +++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java @@ -144,6 +144,8 @@ public class NodeControllerService extends AbstractRemoteService { private final MemoryManager memoryManager; + private boolean shuttedDown = false; + public NodeControllerService(NCConfig ncConfig) throws Exception { this.ncConfig = ncConfig; id = ncConfig.nodeId; @@ -278,6 +280,9 @@ public void start() throws Exception { if (ncAppEntryPoint != null) { ncAppEntryPoint.notifyStartupComplete(); } + + //add JVM shutdown hook + Runtime.getRuntime().addShutdownHook(new JVMShutdownHook(this)); } private void startApplication() throws Exception { @@ -294,16 +299,21 @@ private void startApplication() throws Exception { } @Override - public void stop() throws Exception { - LOGGER.log(Level.INFO, "Stopping NodeControllerService"); - executor.shutdownNow(); - partitionManager.close(); - datasetPartitionManager.close(); - heartbeatTask.cancel(); - netManager.stop(); - datasetNetworkManager.stop(); - queue.stop(); - LOGGER.log(Level.INFO, "Stopped NodeControllerService"); + public synchronized void stop() throws Exception { + if (!shuttedDown) { + LOGGER.log(Level.INFO, "Stopping NodeControllerService"); + executor.shutdownNow(); + partitionManager.close(); + datasetPartitionManager.close(); + heartbeatTask.cancel(); + netManager.stop(); + datasetNetworkManager.stop(); + queue.stop(); + if (ncAppEntryPoint != null) + ncAppEntryPoint.stop(); + LOGGER.log(Level.INFO, "Stopped NodeControllerService"); + shuttedDown = true; + } } public String getId() { @@ -525,4 +535,29 @@ public void sendApplicationMessageToCC(byte[] data, DeploymentId deploymentId, S public IDatasetPartitionManager getDatasetPartitionManager() { return datasetPartitionManager; } -} \ No newline at end of file + + /** + * Shutdown hook that invokes {@link NCApplicationEntryPoint#stop() stop} method. + */ + private static class JVMShutdownHook extends Thread { + + private final NodeControllerService nodeControllerService; + + public JVMShutdownHook(NodeControllerService ncAppEntryPoint) { + this.nodeControllerService = ncAppEntryPoint; + } + + public void run() { + if (LOGGER.isLoggable(Level.INFO)) { + LOGGER.info("Shutdown hook in progress"); + } + try { + nodeControllerService.stop(); + } catch (Exception e) { + if (LOGGER.isLoggable(Level.WARNING)) { + LOGGER.warning("Exception in executing shutdown hook" + e); + } + } + } + } +} diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java index b83e22b57..6c4af5bd8 100644 --- a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java +++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupWriter.java @@ -163,6 +163,7 @@ public void close() throws HyracksDataException { FrameUtils.flushFrame(outFrame, writer); } } + aggregator.close(); aggregateState.close(); writer.close(); } diff --git a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java index fe25db8c8..19200ee7e 100644 --- a/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java +++ b/hyracks/hyracks-storage-am-bloomfilter/src/main/java/edu/uci/ics/hyracks/storage/am/bloomfilter/impls/BloomFilter.java @@ -133,7 +133,7 @@ public synchronized void create() throws HyracksDataException { metaPage.getBuffer().putInt(NUM_HASHES_USED_OFFSET, 0); metaPage.getBuffer().putLong(NUM_ELEMENTS_OFFSET, 0L); metaPage.getBuffer().putLong(NUM_BITS_OFFSET, 0L); - metaPage.releaseWriteLatch(); + metaPage.releaseWriteLatch(true); bufferCache.unpin(metaPage); bufferCache.closeFile(fileId); } @@ -249,7 +249,7 @@ private void persistBloomFilterMetaData() throws HyracksDataException { metaPage.getBuffer().putInt(NUM_HASHES_USED_OFFSET, numHashes); metaPage.getBuffer().putLong(NUM_ELEMENTS_OFFSET, numElements); metaPage.getBuffer().putLong(NUM_BITS_OFFSET, numBits); - metaPage.releaseWriteLatch(); + metaPage.releaseWriteLatch(true); bufferCache.unpin(metaPage); } @@ -273,7 +273,7 @@ public void add(ITupleReference tuple) throws IndexException, HyracksDataExcepti public void end() throws HyracksDataException, IndexException { for (int i = 0; i < numPages; ++i) { ICachedPage page = bloomFilterPages.get(i); - page.releaseWriteLatch(); + page.releaseWriteLatch(true); } } diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java index 67392ac87..cdaf144b1 100644 --- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java +++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMInteriorFrame.java @@ -185,45 +185,50 @@ public void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey s ByteBuffer right = rightFrame.getBuffer(); int tupleCount = getTupleCount(); - // Find split point, and determine into which frame the new tuple should be inserted into. - int tuplesToLeft; + // Find split point, and determine into which frame the new tuple should be inserted into. ITreeIndexFrame targetFrame = null; - - int totalSize = 0; - int halfPageSize = (buf.capacity() - getPageHeaderSize()) / 2; - int i; - for (i = 0; i < tupleCount; ++i) { - frameTuple.resetByTupleIndex(this, i); - totalSize += tupleWriter.bytesRequired(frameTuple) + childPtrSize + slotManager.getSlotSize(); - if (totalSize >= halfPageSize) { - break; - } - } - + frameTuple.resetByTupleIndex(this, tupleCount - 1); + int tuplesToLeft; if (cmp.compare(tuple, frameTuple) > 0) { - tuplesToLeft = i; + // This is a special optimization case when the tuple to be inserted is the largest key on the page. targetFrame = rightFrame; + tuplesToLeft = tupleCount; } else { - tuplesToLeft = i + 1; - targetFrame = this; - } - int tuplesToRight = tupleCount - tuplesToLeft; - - // Copy entire page. - System.arraycopy(buf.array(), 0, right.array(), 0, buf.capacity()); - - // On the right page we need to copy rightmost slots to left. - int src = rightFrame.getSlotManager().getSlotEndOff(); - int dest = rightFrame.getSlotManager().getSlotEndOff() + tuplesToLeft - * rightFrame.getSlotManager().getSlotSize(); - int length = rightFrame.getSlotManager().getSlotSize() * tuplesToRight; - System.arraycopy(right.array(), src, right.array(), dest, length); - right.putInt(tupleCountOff, tuplesToRight); - - // On the left page, remove the highest key and make its child pointer - // the rightmost child pointer. - buf.putInt(tupleCountOff, tuplesToLeft); + int totalSize = 0; + int halfPageSize = (buf.capacity() - getPageHeaderSize()) / 2; + int i; + for (i = 0; i < tupleCount; ++i) { + frameTuple.resetByTupleIndex(this, i); + totalSize += tupleWriter.bytesRequired(frameTuple) + childPtrSize + slotManager.getSlotSize(); + if (totalSize >= halfPageSize) { + break; + } + } + if (cmp.compare(tuple, frameTuple) > 0) { + tuplesToLeft = i; + targetFrame = rightFrame; + } else { + tuplesToLeft = i + 1; + targetFrame = this; + } + int tuplesToRight = tupleCount - tuplesToLeft; + + // Copy entire page. + System.arraycopy(buf.array(), 0, right.array(), 0, buf.capacity()); + + // On the right page we need to copy rightmost slots to left. + int src = rightFrame.getSlotManager().getSlotEndOff(); + int dest = rightFrame.getSlotManager().getSlotEndOff() + tuplesToLeft + * rightFrame.getSlotManager().getSlotSize(); + int length = rightFrame.getSlotManager().getSlotSize() * tuplesToRight; + System.arraycopy(right.array(), src, right.array(), dest, length); + right.putInt(tupleCountOff, tuplesToRight); + + // On the left page, remove the highest key and make its child pointer + // the rightmost child pointer. + buf.putInt(tupleCountOff, tuplesToLeft); + } // Copy the split key to be inserted. // We must do so because setting the new split key will overwrite the // old split key, and we cannot insert the existing split key at this point. @@ -245,7 +250,6 @@ public void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey s // Compact both pages. rightFrame.compact(); compact(); - // Insert the saved split key. int targetTupleIndex; // it's safe to catch this exception since it will have been caught before reaching here diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java index 187cd52a9..995e8a79b 100644 --- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java +++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/BTreeNSMLeafFrame.java @@ -49,7 +49,7 @@ public BTreeNSMLeafFrame(ITreeIndexTupleWriter tupleWriter) { public int getBytesRequriedToWriteTuple(ITupleReference tuple) { return tupleWriter.bytesRequired(tuple) + slotManager.getSlotSize(); } - + @Override public void initBuffer(byte level) { super.initBuffer(level); @@ -148,45 +148,51 @@ public void split(ITreeIndexFrame rightFrame, ITupleReference tuple, ISplitKey s // Find split point, and determine into which frame the new tuple should // be inserted into. - int tuplesToLeft; ITreeIndexFrame targetFrame = null; - int totalSize = 0; - int halfPageSize = (buf.capacity() - getPageHeaderSize()) / 2; - int i; - for (i = 0; i < tupleCount; ++i) { - frameTuple.resetByTupleIndex(this, i); - totalSize += tupleWriter.getCopySpaceRequired(frameTuple) + slotManager.getSlotSize(); - if (totalSize >= halfPageSize) { - break; - } - } - - if (cmp.compare(tuple, frameTuple) >= 0) { - tuplesToLeft = i + 1; + frameTuple.resetByTupleIndex(this, tupleCount - 1); + if (cmp.compare(tuple, frameTuple) > 0) { + // This is a special optimization case when the tuple to be inserted is the largest key on the page. targetFrame = rightFrame; } else { - tuplesToLeft = i; - targetFrame = this; - } - int tuplesToRight = tupleCount - tuplesToLeft; + int tuplesToLeft; + int totalSize = 0; + int halfPageSize = (buf.capacity() - getPageHeaderSize()) / 2; + int i; + for (i = 0; i < tupleCount; ++i) { + frameTuple.resetByTupleIndex(this, i); + totalSize += tupleWriter.getCopySpaceRequired(frameTuple) + slotManager.getSlotSize(); + if (totalSize >= halfPageSize) { + break; + } + } - // Copy entire page. - System.arraycopy(buf.array(), 0, right.array(), 0, buf.capacity()); + if (cmp.compare(tuple, frameTuple) >= 0) { + tuplesToLeft = i + 1; + targetFrame = rightFrame; + } else { + tuplesToLeft = i; + targetFrame = this; + } + int tuplesToRight = tupleCount - tuplesToLeft; - // On the right page we need to copy rightmost slots to the left. - int src = rightFrame.getSlotManager().getSlotEndOff(); - int dest = rightFrame.getSlotManager().getSlotEndOff() + tuplesToLeft - * rightFrame.getSlotManager().getSlotSize(); - int length = rightFrame.getSlotManager().getSlotSize() * tuplesToRight; - System.arraycopy(right.array(), src, right.array(), dest, length); - right.putInt(tupleCountOff, tuplesToRight); + // Copy entire page. + System.arraycopy(buf.array(), 0, right.array(), 0, buf.capacity()); - // On left page only change the tupleCount indicator. - buf.putInt(tupleCountOff, tuplesToLeft); + // On the right page we need to copy rightmost slots to the left. + int src = rightFrame.getSlotManager().getSlotEndOff(); + int dest = rightFrame.getSlotManager().getSlotEndOff() + tuplesToLeft + * rightFrame.getSlotManager().getSlotSize(); + int length = rightFrame.getSlotManager().getSlotSize() * tuplesToRight; + System.arraycopy(right.array(), src, right.array(), dest, length); + right.putInt(tupleCountOff, tuplesToRight); - // Compact both pages. - rightFrame.compact(); - compact(); + // On left page only change the tupleCount indicator. + buf.putInt(tupleCountOff, tuplesToLeft); + + // Compact both pages. + rightFrame.compact(); + compact(); + } // Insert the new tuple. int targetTupleIndex; diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/OrderedSlotManager.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/OrderedSlotManager.java index 2071a779b..2316c6b38 100644 --- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/OrderedSlotManager.java +++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/frames/OrderedSlotManager.java @@ -27,19 +27,29 @@ public class OrderedSlotManager extends AbstractSlotManager { @Override public int findTupleIndex(ITupleReference searchKey, ITreeIndexTupleReference frameTuple, MultiComparator multiCmp, FindTupleMode mode, FindTupleNoExactMatchPolicy matchPolicy) { - if (frame.getTupleCount() <= 0) { + int tupleCount = frame.getTupleCount(); + if (tupleCount <= 0) { return GREATEST_KEY_INDICATOR; } int mid; - int begin = 0; - int end = frame.getTupleCount() - 1; + int begin; + int end = tupleCount - 1; + + frameTuple.resetByTupleIndex(frame, end); + int cmp = multiCmp.compare(searchKey, frameTuple); + if (cmp > 0) { + // This is a special optimization case when the tuple to be searched is larger than all the keys on the page. + begin = tupleCount; + } else { + begin = 0; + } while (begin <= end) { mid = (begin + end) / 2; frameTuple.resetByTupleIndex(frame, mid); - int cmp = multiCmp.compare(searchKey, frameTuple); + cmp = multiCmp.compare(searchKey, frameTuple); if (cmp < 0) { end = mid - 1; } else if (cmp > 0) { @@ -66,7 +76,7 @@ public int findTupleIndex(ITupleReference searchKey, ITreeIndexTupleReference fr } if (matchPolicy == FindTupleNoExactMatchPolicy.HIGHER_KEY) { - if (begin > frame.getTupleCount() - 1) { + if (begin > tupleCount - 1) { return GREATEST_KEY_INDICATOR; } frameTuple.resetByTupleIndex(frame, begin); diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java index ff94040ed..fef0f8154 100644 --- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java +++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTree.java @@ -217,7 +217,7 @@ private void unsetSmPages(BTreeOpContext ctx) throws HyracksDataException { ctx.interiorFrame.setPage(smPage); ctx.interiorFrame.setSmFlag(false); } finally { - smPage.releaseWriteLatch(); + smPage.releaseWriteLatch(true); bufferCache.unpin(smPage); } } @@ -261,11 +261,11 @@ private void createNewRoot(BTreeOpContext ctx) throws HyracksDataException, Tree int targetTupleIndex = ctx.interiorFrame.findInsertTupleIndex(ctx.splitKey.getTuple()); ctx.interiorFrame.insert(ctx.splitKey.getTuple(), targetTupleIndex); } finally { - newLeftNode.releaseWriteLatch(); + newLeftNode.releaseWriteLatch(true); bufferCache.unpin(newLeftNode); } } finally { - leftNode.releaseWriteLatch(); + leftNode.releaseWriteLatch(true); bufferCache.unpin(leftNode); } } @@ -447,7 +447,7 @@ private boolean performLeafSplit(int pageId, ITupleReference tuple, BTreeOpConte treeLatch.writeLock().unlock(); throw e; } finally { - rightNode.releaseWriteLatch(); + rightNode.releaseWriteLatch(true); bufferCache.unpin(rightNode); } return false; @@ -533,7 +533,7 @@ private void insertInterior(ICachedPage node, int pageId, ITupleReference tuple, ctx.splitKey.setPages(pageId, rightPageId); } finally { - rightNode.releaseWriteLatch(); + rightNode.releaseWriteLatch(true); bufferCache.unpin(rightNode); } break; @@ -616,7 +616,7 @@ private void performOp(int pageId, ICachedPage parent, boolean parentIsReadLatch if (parentIsReadLatched) { parent.releaseReadLatch(); } else { - parent.releaseWriteLatch(); + parent.releaseWriteLatch(true); } bufferCache.unpin(parent); } @@ -664,7 +664,7 @@ private void performOp(int pageId, ICachedPage parent, boolean parentIsReadLatch // Insert or update op. Both can cause split keys to propagate upwards. insertInterior(interiorNode, pageId, ctx.splitKey.getTuple(), ctx); } finally { - interiorNode.releaseWriteLatch(); + interiorNode.releaseWriteLatch(true); bufferCache.unpin(interiorNode); } } else { @@ -694,7 +694,7 @@ private void performOp(int pageId, ICachedPage parent, boolean parentIsReadLatch if (isReadLatched) { node.releaseReadLatch(); } else { - node.releaseWriteLatch(); + node.releaseWriteLatch(true); } bufferCache.unpin(node); @@ -747,7 +747,7 @@ private void performOp(int pageId, ICachedPage parent, boolean parentIsReadLatch } } if (ctx.op != IndexOperation.SEARCH) { - node.releaseWriteLatch(); + node.releaseWriteLatch(true); bufferCache.unpin(node); } if (restartOp) { @@ -764,7 +764,7 @@ private void performOp(int pageId, ICachedPage parent, boolean parentIsReadLatch if (isReadLatched) { node.releaseReadLatch(); } else { - node.releaseWriteLatch(); + node.releaseWriteLatch(true); } bufferCache.unpin(node); ctx.exceptionHandled = true; @@ -777,7 +777,7 @@ private void performOp(int pageId, ICachedPage parent, boolean parentIsReadLatch if (isReadLatched) { node.releaseReadLatch(); } else { - node.releaseWriteLatch(); + node.releaseWriteLatch(true); } bufferCache.unpin(node); } @@ -897,9 +897,9 @@ public void upsertIfConditionElseInsert(ITupleReference tuple, ITupleAcceptor ac } @Override - public ITreeIndexCursor createSearchCursor() { + public ITreeIndexCursor createSearchCursor(boolean exclusive) { IBTreeLeafFrame leafFrame = (IBTreeLeafFrame) btree.getLeafFrameFactory().createFrame(); - return new BTreeRangeSearchCursor(leafFrame, false); + return new BTreeRangeSearchCursor(leafFrame, exclusive); } @Override @@ -990,7 +990,7 @@ public void add(ITupleReference tuple) throws IndexException, HyracksDataExcepti leafFrontier.pageId = freePageManager.getFreePage(metaFrame); ((IBTreeLeafFrame) leafFrame).setNextLeaf(leafFrontier.pageId); - leafFrontier.page.releaseWriteLatch(); + leafFrontier.page.releaseWriteLatch(true); bufferCache.unpin(leafFrontier.page); splitKey.setRightPage(leafFrontier.pageId); @@ -1062,7 +1062,7 @@ protected void propagateBulk(int level) throws HyracksDataException { ((IBTreeInteriorFrame) interiorFrame).deleteGreatest(); - frontier.page.releaseWriteLatch(); + frontier.page.releaseWriteLatch(true); bufferCache.unpin(frontier.page); frontier.pageId = freePageManager.getFreePage(metaFrame); diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java index ee65a4600..38c16246a 100644 --- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java +++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeCountingSearchCursor.java @@ -51,6 +51,7 @@ public class BTreeCountingSearchCursor implements ITreeIndexCursor { private final IBTreeLeafFrame frame; private final ITreeIndexTupleReference frameTuple; private final boolean exclusiveLatchNodes; + private boolean isPageDirty; private RangePredicate pred; private MultiComparator lowKeyCmp; @@ -61,8 +62,8 @@ public class BTreeCountingSearchCursor implements ITreeIndexCursor { // For storing the count. private byte[] countBuf = new byte[4]; private ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(1); - private ArrayTupleReference countTuple = new ArrayTupleReference(); - + private ArrayTupleReference countTuple = new ArrayTupleReference(); + public BTreeCountingSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes) { this.frame = frame; this.frameTuple = frame.createTupleReference(); @@ -74,7 +75,7 @@ public void open(ICursorInitialState initialState, ISearchPredicate searchPred) // in case open is called multiple times without closing if (page != null) { if (exclusiveLatchNodes) { - page.releaseWriteLatch(); + page.releaseWriteLatch(isPageDirty); } else { page.releaseReadLatch(); } @@ -82,6 +83,7 @@ public void open(ICursorInitialState initialState, ISearchPredicate searchPred) } page = ((BTreeCursorInitialState) initialState).getPage(); + isPageDirty = false; frame.setPage(page); pred = (RangePredicate) searchPred; @@ -107,7 +109,7 @@ public void open(ICursorInitialState initialState, ISearchPredicate searchPred) } tupleIndex = getLowKeyIndex(); - stopTupleIndex = getHighKeyIndex(); + stopTupleIndex = getHighKeyIndex(); } private void fetchNextLeafPage(int nextLeafPage) throws HyracksDataException { @@ -115,13 +117,14 @@ private void fetchNextLeafPage(int nextLeafPage) throws HyracksDataException { ICachedPage nextLeaf = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeafPage), false); if (exclusiveLatchNodes) { nextLeaf.acquireWriteLatch(); - page.releaseWriteLatch(); + page.releaseWriteLatch(isPageDirty); } else { nextLeaf.acquireReadLatch(); page.releaseReadLatch(); } bufferCache.unpin(page); page = nextLeaf; + isPageDirty = false; frame.setPage(page); nextLeafPage = frame.getNextLeaf(); } while (frame.getTupleCount() == 0 && nextLeafPage > 0); @@ -199,7 +202,7 @@ public void next() throws HyracksDataException { public void close() throws HyracksDataException { if (page != null) { if (exclusiveLatchNodes) { - page.releaseWriteLatch(); + page.releaseWriteLatch(isPageDirty); } else { page.releaseReadLatch(); } @@ -208,6 +211,7 @@ public void close() throws HyracksDataException { tupleBuilder.reset(); tupleIndex = 0; page = null; + isPageDirty = false; pred = null; count = -1; } @@ -218,7 +222,7 @@ public void reset() { close(); } catch (Exception e) { e.printStackTrace(); - } + } } @Override @@ -246,4 +250,13 @@ public boolean exclusiveLatchNodes() { return exclusiveLatchNodes; } + @Override + public void markCurrentTupleAsUpdated() throws HyracksDataException { + if (exclusiveLatchNodes) { + isPageDirty = true; + } else { + throw new HyracksDataException("This cursor has not been created with the intention to allow updates."); + } + } + } diff --git a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java index 4546b0e89..584af4ec9 100644 --- a/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java +++ b/hyracks/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/impls/BTreeRangeSearchCursor.java @@ -40,6 +40,7 @@ public class BTreeRangeSearchCursor implements ITreeIndexCursor { private final IBTreeLeafFrame frame; private final ITreeIndexTupleReference frameTuple; private final boolean exclusiveLatchNodes; + private boolean isPageDirty; private IBufferCache bufferCache = null; private int fileId = -1; @@ -80,7 +81,7 @@ public BTreeRangeSearchCursor(IBTreeLeafFrame frame, boolean exclusiveLatchNodes public void close() throws HyracksDataException { if (page != null) { if (exclusiveLatchNodes) { - page.releaseWriteLatch(); + page.releaseWriteLatch(isPageDirty); } else { page.releaseReadLatch(); } @@ -89,6 +90,7 @@ public void close() throws HyracksDataException { tupleIndex = 0; page = null; + isPageDirty = false; pred = null; } @@ -114,7 +116,7 @@ private void fetchNextLeafPage(int nextLeafPage) throws HyracksDataException { ICachedPage nextLeaf = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, nextLeafPage), false); if (exclusiveLatchNodes) { nextLeaf.acquireWriteLatch(); - page.releaseWriteLatch(); + page.releaseWriteLatch(isPageDirty); } else { nextLeaf.acquireReadLatch(); page.releaseReadLatch(); @@ -122,6 +124,7 @@ private void fetchNextLeafPage(int nextLeafPage) throws HyracksDataException { bufferCache.unpin(page); page = nextLeaf; + isPageDirty = false; frame.setPage(page); pageId = nextLeafPage; nextLeafPage = frame.getNextLeaf(); @@ -163,12 +166,13 @@ public boolean hasNext() throws HyracksDataException { // unlatch/unpin if (exclusiveLatchNodes) { - page.releaseWriteLatch(); + page.releaseWriteLatch(isPageDirty); } else { page.releaseReadLatch(); } bufferCache.unpin(page); page = null; + isPageDirty = false; // reconcile searchCb.reconcile(reconciliationTuple); @@ -240,7 +244,7 @@ public void open(ICursorInitialState initialState, ISearchPredicate searchPred) // in case open is called multiple times without closing if (page != null) { if (exclusiveLatchNodes) { - page.releaseWriteLatch(); + page.releaseWriteLatch(isPageDirty); } else { page.releaseReadLatch(); } @@ -251,6 +255,7 @@ public void open(ICursorInitialState initialState, ISearchPredicate searchPred) originalKeyCmp = initialState.getOriginalKeyComparator(); pageId = ((BTreeCursorInitialState) initialState).getPageId(); page = initialState.getPage(); + isPageDirty = false; frame.setPage(page); pred = (RangePredicate) searchPred; @@ -300,4 +305,13 @@ public void setFileId(int fileId) { public boolean exclusiveLatchNodes() { return exclusiveLatchNodes; } + + @Override + public void markCurrentTupleAsUpdated() throws HyracksDataException { + if (exclusiveLatchNodes) { + isPageDirty = true; + } else { + throw new HyracksDataException("This cursor has not been created with the intention to allow updates."); + } + } } \ No newline at end of file diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexAccessor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexAccessor.java index 5eb3cddff..a9aaad880 100644 --- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexAccessor.java +++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/IIndexAccessor.java @@ -83,7 +83,7 @@ public interface IIndexAccessor { * Creates a cursor appropriate for passing into search(). * */ - public IIndexCursor createSearchCursor(); + public IIndexCursor createSearchCursor(boolean exclusive); /** * Open the given cursor for an index search using the given predicate as diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java index 5102d27fe..a097425bd 100644 --- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java +++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/api/ITreeIndexCursor.java @@ -15,17 +15,20 @@ package edu.uci.ics.hyracks.storage.am.common.api; +import edu.uci.ics.hyracks.api.exceptions.HyracksDataException; import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache; import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage; public interface ITreeIndexCursor extends IIndexCursor { - public ICachedPage getPage(); + public ICachedPage getPage(); - public void setBufferCache(IBufferCache bufferCache); + public void setBufferCache(IBufferCache bufferCache); - public void setFileId(int fileId); + public void setFileId(int fileId); - // For allowing updates. - public boolean exclusiveLatchNodes(); + // For allowing updates. + public boolean exclusiveLatchNodes(); + + public void markCurrentTupleAsUpdated() throws HyracksDataException; } diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java index 83fb5ee06..2696af918 100644 --- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java +++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java @@ -70,7 +70,7 @@ public IndexSearchOperatorNodePushable(IIndexOperatorDescriptor opDesc, IHyracks protected abstract void resetSearchPredicate(int tupleIndex); protected IIndexCursor createCursor() { - return indexAccessor.createSearchCursor(); + return indexAccessor.createSearchCursor(false); } @Override diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/freepage/LinkedListFreePageManager.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/freepage/LinkedListFreePageManager.java index 2e25f4cbe..a51ce98da 100644 --- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/freepage/LinkedListFreePageManager.java +++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/freepage/LinkedListFreePageManager.java @@ -77,14 +77,14 @@ public void addFreePage(ITreeIndexMetaDataFrame metaFrame, int freePage) metaFrame.setMaxPage(metaMaxPage); metaFrame.addFreePage(freePage); } finally { - newNode.releaseWriteLatch(); + newNode.releaseWriteLatch(true); bufferCache.unpin(newNode); } } } catch (Exception e) { e.printStackTrace(); } finally { - metaNode.releaseWriteLatch(); + metaNode.releaseWriteLatch(true); bufferCache.unpin(metaNode); } } @@ -136,7 +136,7 @@ public int getFreePage(ITreeIndexMetaDataFrame metaFrame) metaFrame.addFreePage(nextPage); } } finally { - nextNode.releaseWriteLatch(); + nextNode.releaseWriteLatch(true); bufferCache.unpin(nextNode); } } else { @@ -146,7 +146,7 @@ public int getFreePage(ITreeIndexMetaDataFrame metaFrame) } } } finally { - metaNode.releaseWriteLatch(); + metaNode.releaseWriteLatch(true); bufferCache.unpin(metaNode); } @@ -164,7 +164,7 @@ public int getMaxPage(ITreeIndexMetaDataFrame metaFrame) metaFrame.setPage(metaNode); maxPage = metaFrame.getMaxPage(); } finally { - metaNode.releaseWriteLatch(); + metaNode.releaseWriteLatch(true); bufferCache.unpin(metaNode); } return maxPage; @@ -183,7 +183,7 @@ public void init(ITreeIndexMetaDataFrame metaFrame, int currentMaxPage) metaFrame.initBuffer(META_PAGE_LEVEL_INDICATOR); metaFrame.setMaxPage(currentMaxPage); } finally { - metaNode.releaseWriteLatch(); + metaNode.releaseWriteLatch(true); bufferCache.unpin(metaNode); } } diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java index 3e4ecf80a..e191fd05e 100644 --- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java +++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/AbstractTreeIndex.java @@ -111,7 +111,7 @@ private void initEmptyTree() throws HyracksDataException { frame.setPage(rootNode); frame.initBuffer((byte) 0); } finally { - rootNode.releaseWriteLatch(); + rootNode.releaseWriteLatch(true); bufferCache.unpin(rootNode); } } @@ -293,7 +293,7 @@ public AbstractTreeIndexBulkLoader(float fillFactor) throws TreeIndexException, protected void handleException() throws HyracksDataException { // Unlatch and unpin pages. for (NodeFrontier nodeFrontier : nodeFrontiers) { - nodeFrontier.page.releaseWriteLatch(); + nodeFrontier.page.releaseWriteLatch(true); bufferCache.unpin(nodeFrontier.page); } releasedLatches = true; @@ -309,7 +309,7 @@ public void end() throws HyracksDataException { System.arraycopy(lastNodeFrontier.page.getBuffer().array(), 0, newRoot.getBuffer().array(), 0, lastNodeFrontier.page.getBuffer().capacity()); } finally { - newRoot.releaseWriteLatch(); + newRoot.releaseWriteLatch(true); bufferCache.unpin(newRoot); // register old root as a free page @@ -318,7 +318,7 @@ public void end() throws HyracksDataException { if (!releasedLatches) { for (int i = 0; i < nodeFrontiers.size(); i++) { try { - nodeFrontiers.get(i).page.releaseWriteLatch(); + nodeFrontiers.get(i).page.releaseWriteLatch(true); } catch (Exception e) { //ignore illegal monitor state exception } diff --git a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java index 0f97b0d6a..759234850 100644 --- a/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java +++ b/hyracks/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/impls/TreeIndexDiskOrderScanCursor.java @@ -147,4 +147,9 @@ public void setMaxPageId(int maxPageId) { public boolean exclusiveLatchNodes() { return false; } + + @Override + public void markCurrentTupleAsUpdated() throws HyracksDataException { + throw new HyracksDataException("Updating tuples is not supported with this cursor."); + } } diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java index 28f44e6e3..cb4d26888 100644 --- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java +++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTree.java @@ -404,7 +404,7 @@ public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataExceptio IIndexBulkLoader builder = component.getBloomFilter().createBuilder(numElements, bloomFilterSpec.getNumHashes(), bloomFilterSpec.getNumBucketsPerElements()); - IIndexCursor scanCursor = accessor.createSearchCursor(); + IIndexCursor scanCursor = accessor.createSearchCursor(false); accessor.search(scanCursor, nullPred); try { while (scanCursor.hasNext()) { @@ -622,7 +622,7 @@ public LSMBTreeAccessor(ILSMHarness lsmHarness, ILSMIndexOperationContext ctx) { } @Override - public IIndexCursor createSearchCursor() { + public IIndexCursor createSearchCursor(boolean exclusive) { return new LSMBTreeSearchCursor(ctx); } diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java index fc09a7432..7aba7b3e9 100644 --- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java +++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreePointSearchCursor.java @@ -219,4 +219,9 @@ public void setFileId(int fileId) { public boolean exclusiveLatchNodes() { return false; } + + @Override + public void markCurrentTupleAsUpdated() throws HyracksDataException { + throw new HyracksDataException("Updating tuples is not supported with this cursor."); + } } \ No newline at end of file diff --git a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java index 366fc1044..dd99833e7 100644 --- a/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java +++ b/hyracks/hyracks-storage-am-lsm-btree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/btree/impls/LSMBTreeSearchCursor.java @@ -129,4 +129,8 @@ public boolean exclusiveLatchNodes() { return currentCursor.exclusiveLatchNodes(); } + @Override + public void markCurrentTupleAsUpdated() throws HyracksDataException { + throw new HyracksDataException("Updating tuples is not supported with this cursor."); + } } \ No newline at end of file diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java index 44bcfc255..3d7512f23 100644 --- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java +++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/AbstractLSMIndex.java @@ -125,16 +125,25 @@ protected void markAsValidInternal(ITreeIndex treeIndex) throws HyracksDataExcep try { metadataFrame.setPage(metadataPage); metadataFrame.setValid(true); - - // Flush the single modified page to disk. - bufferCache.flushDirtyPage(metadataPage); - - // Force modified metadata page to disk. - bufferCache.force(fileId, true); } finally { - metadataPage.releaseWriteLatch(); + metadataPage.releaseWriteLatch(true); bufferCache.unpin(metadataPage); } + + // WARNING: flushing the metadata page should be done after releasing the write latch; otherwise, the page + // won't be flushed to disk because it won't be dirty until the write latch has been released. + metadataPage = bufferCache.tryPin(BufferedFileHandle.getDiskPageId(fileId, metadataPageId)); + if (metadataPage != null) { + try { + // Flush the single modified page to disk. + bufferCache.flushDirtyPage(metadataPage); + } finally { + bufferCache.unpin(metadataPage); + } + } + + // Force modified metadata page to disk. + bufferCache.force(fileId, true); } @Override diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java index 2bc45a9cf..5a4699e53 100644 --- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java +++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java @@ -263,4 +263,9 @@ protected void setPriorityQueueComparator() { protected int compare(MultiComparator cmp, ITupleReference tupleA, ITupleReference tupleB) { return cmp.compare(tupleA, tupleB); } + + @Override + public void markCurrentTupleAsUpdated() throws HyracksDataException { + throw new HyracksDataException("Updating tuples is not supported with this cursor."); + } } \ No newline at end of file diff --git a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java index bd3b2783d..7f804c105 100644 --- a/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java +++ b/hyracks/hyracks-storage-am-lsm-common/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/common/impls/VirtualBufferCache.java @@ -336,7 +336,7 @@ public void acquireWriteLatch() { } @Override - public void releaseWriteLatch() { + public void releaseWriteLatch(boolean markDirty) { latch.writeLock().unlock(); } diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java index 93c5a5850..7b4ae4223 100644 --- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java +++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndex.java @@ -458,7 +458,7 @@ public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataExceptio .getInvIndex().createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE); BTreeAccessor memBTreeAccessor = memInvIndexAccessor.getBTreeAccessor(); RangePredicate nullPred = new RangePredicate(null, null, true, true, null, null); - IIndexCursor scanCursor = memBTreeAccessor.createSearchCursor(); + IIndexCursor scanCursor = memBTreeAccessor.createSearchCursor(false); memBTreeAccessor.search(scanCursor, nullPred); // Bulk load the disk inverted index from the in-memory inverted index. @@ -498,7 +498,7 @@ public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataExceptio BTree diskDeletedKeysBTree = component.getDeletedKeysBTree(); // Create a scan cursor on the deleted keys BTree underlying the in-memory inverted index. - IIndexCursor deletedKeysScanCursor = deletedKeysBTreeAccessor.createSearchCursor(); + IIndexCursor deletedKeysScanCursor = deletedKeysBTreeAccessor.createSearchCursor(false); deletedKeysBTreeAccessor.search(deletedKeysScanCursor, nullPred); // Bulk load the deleted-keys BTree. diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java index 7e34dfe72..f76085a2c 100644 --- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java +++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexAccessor.java @@ -71,7 +71,7 @@ public void search(IIndexCursor cursor, ISearchPredicate searchPred) throws Hyra lsmHarness.search(ctx, cursor, searchPred); } - public IIndexCursor createSearchCursor() { + public IIndexCursor createSearchCursor(boolean exclusive) { return new LSMInvertedIndexSearchCursor(); } diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java index 3efaabad3..9eb317dec 100644 --- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java +++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexDeletedKeysBTreeMergeCursor.java @@ -54,7 +54,7 @@ public void open(ICursorInitialState initialState, ISearchPredicate searchPred) RangePredicate btreePredicate = new RangePredicate(null, null, true, true, keyCmp, keyCmp); ArrayList btreeAccessors = lsmInitialState.getDeletedKeysBTreeAccessors(); for (int i = 0; i < numBTrees; i++) { - rangeCursors[i] = btreeAccessors.get(i).createSearchCursor(); + rangeCursors[i] = btreeAccessors.get(i).createSearchCursor(false); btreeAccessors.get(i).search(rangeCursors[i], btreePredicate); } setPriorityQueueComparator(); diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java index a6ff07ec2..48e94eb0b 100644 --- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java +++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexRangeSearchCursor.java @@ -77,7 +77,7 @@ public void open(ICursorInitialState initState, ISearchPredicate searchPred) thr ILSMComponent component = operationalComponents.get(i); if (component.getType() == LSMComponentType.MEMORY) { // No need for a bloom filter for the in-memory BTree. - deletedKeysBTreeCursors[i] = deletedKeysBTreeAccessors.get(i).createSearchCursor(); + deletedKeysBTreeCursors[i] = deletedKeysBTreeAccessors.get(i).createSearchCursor(false); } else { deletedKeysBTreeCursors[i] = new BloomFilterAwareBTreePointSearchCursor( (IBTreeLeafFrame) lsmInitState.getgetDeletedKeysBTreeLeafFrameFactory().createFrame(), diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java index 19acbfb8e..8ceedaff9 100644 --- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java +++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/impls/LSMInvertedIndexSearchCursor.java @@ -76,7 +76,7 @@ public void open(ICursorInitialState initialState, ISearchPredicate searchPred) ILSMComponent component = operationalComponents.get(i); if (component.getType() == LSMComponentType.MEMORY) { // No need for a bloom filter for the in-memory BTree. - deletedKeysBTreeCursors[i] = deletedKeysBTreeAccessors.get(i).createSearchCursor(); + deletedKeysBTreeCursors[i] = deletedKeysBTreeAccessors.get(i).createSearchCursor(false); } else { deletedKeysBTreeCursors[i] = new BloomFilterAwareBTreePointSearchCursor((IBTreeLeafFrame) lsmInitState .getgetDeletedKeysBTreeLeafFrameFactory().createFrame(), false, @@ -134,7 +134,7 @@ public boolean hasNext() throws HyracksDataException, IndexException { while (accessorIndex < indexAccessors.size()) { // Current cursor has been exhausted, switch to next accessor/cursor. currentAccessor = indexAccessors.get(accessorIndex); - currentCursor = currentAccessor.createSearchCursor(); + currentCursor = currentAccessor.createSearchCursor(false); try { currentAccessor.search(currentCursor, searchPred); } catch (OccurrenceThresholdPanicException e) { diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java index 7e9b483fb..ef8af320a 100644 --- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java +++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedIndexAccessor.java @@ -65,7 +65,7 @@ public void delete(ITupleReference tuple) throws HyracksDataException, IndexExce } @Override - public IIndexCursor createSearchCursor() { + public IIndexCursor createSearchCursor(boolean exclusive) { return new OnDiskInvertedIndexSearchCursor(searcher, index.getInvListTypeTraits().length); } diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java index c742d0ce6..84e0fade0 100644 --- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java +++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/inmemory/InMemoryInvertedListCursor.java @@ -67,7 +67,7 @@ public void prepare(BTreeAccessor btreeAccessor, RangePredicate btreePred, Multi // Avoid object creation if this.btreeAccessor == btreeAccessor. if (this.btreeAccessor != btreeAccessor) { this.btreeAccessor = btreeAccessor; - this.btreeCursor = btreeAccessor.createSearchCursor(); + this.btreeCursor = btreeAccessor.createSearchCursor(false); this.countingCursor = btreeAccessor.createCountingSearchCursor(); this.btreePred = btreePred; this.btreePred.setLowKeyComparator(tokenFieldsCmp); diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java index 4f586f957..d72e019d2 100644 --- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java +++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndex.java @@ -325,7 +325,7 @@ public OnDiskInvertedIndexBulkLoader(float btreeFillFactor, boolean verifyInput, } public void pinNextPage() throws HyracksDataException { - currentPage.releaseWriteLatch(); + currentPage.releaseWriteLatch(true); bufferCache.unpin(currentPage); currentPageId++; currentPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), true); @@ -431,7 +431,7 @@ public void end() throws IndexException, HyracksDataException { btreeBulkloader.end(); if (currentPage != null) { - currentPage.releaseWriteLatch(); + currentPage.releaseWriteLatch(true); bufferCache.unpin(currentPage); } invListsMaxPageId = currentPageId; @@ -484,7 +484,7 @@ protected OnDiskInvertedIndexAccessor(OnDiskInvertedIndex index, IInvertedIndexS } @Override - public IIndexCursor createSearchCursor() { + public IIndexCursor createSearchCursor(boolean exclusive) { return new OnDiskInvertedIndexSearchCursor(searcher, index.getInvListTypeTraits().length); } @@ -590,7 +590,7 @@ public void validate() throws HyracksDataException { // Scan the btree and validate the order of elements in each inverted-list. IIndexAccessor btreeAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE); - IIndexCursor btreeCursor = btreeAccessor.createSearchCursor(); + IIndexCursor btreeCursor = btreeAccessor.createSearchCursor(false); MultiComparator btreeCmp = MultiComparator.createIgnoreFieldLength(btree.getComparatorFactories()); RangePredicate rangePred = new RangePredicate(null, null, true, true, btreeCmp, btreeCmp); int[] fieldPermutation = new int[tokenTypeTraits.length]; diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java index 5c4760165..509409f44 100644 --- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java +++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexOpContext.java @@ -36,7 +36,7 @@ public class OnDiskInvertedIndexOpContext implements IIndexOperationContext { public OnDiskInvertedIndexOpContext(BTree btree) { // TODO: Ignore opcallbacks for now. btreeAccessor = btree.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE); - btreeCursor = btreeAccessor.createSearchCursor(); + btreeCursor = btreeAccessor.createSearchCursor(false); searchCmp = MultiComparator.createIgnoreFieldLength(btree.getComparatorFactories()); if (btree.getComparatorFactories().length > 1) { prefixSearchCmp = MultiComparator.create(btree.getComparatorFactories(), 0, 1); diff --git a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java index defe29cd8..9cb72312b 100644 --- a/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java +++ b/hyracks/hyracks-storage-am-lsm-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/ondisk/OnDiskInvertedIndexRangeSearchCursor.java @@ -59,7 +59,7 @@ public OnDiskInvertedIndexRangeSearchCursor(IInvertedIndex invIndex, IIndexOpera fieldPermutation[i] = i; } tokenTuple = new PermutingTupleReference(fieldPermutation); - btreeCursor = btreeAccessor.createSearchCursor(); + btreeCursor = btreeAccessor.createSearchCursor(false); concatTuple = new ConcatenatingTupleReference(2); invListCursor = invIndex.createInvertedListCursor(); unpinNeeded = false; diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java index 3d2cc62a8..95d1a118c 100644 --- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java +++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTree.java @@ -191,7 +191,7 @@ public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataExceptio // scan the memory RTree ITreeIndexAccessor memRTreeAccessor = flushingComponent.getRTree().createAccessor( NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE); - RTreeSearchCursor rtreeScanCursor = (RTreeSearchCursor) memRTreeAccessor.createSearchCursor(); + RTreeSearchCursor rtreeScanCursor = (RTreeSearchCursor) memRTreeAccessor.createSearchCursor(false); SearchPredicate rtreeNullPredicate = new SearchPredicate(null, null); memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate); LSMRTreeDiskComponent component = createDiskComponent(componentFactory, flushOp.getRTreeFlushTarget(), @@ -258,7 +258,7 @@ public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataExceptio BloomFilterSpecification bloomFilterSpec = BloomCalculations.computeBloomSpec(maxBucketsPerElement, bloomFilterFalsePositiveRate); - IIndexCursor btreeScanCursor = memBTreeAccessor.createSearchCursor(); + IIndexCursor btreeScanCursor = memBTreeAccessor.createSearchCursor(false); memBTreeAccessor.search(btreeScanCursor, btreeNullPredicate); BTree diskBTree = component.getBTree(); @@ -360,7 +360,7 @@ public LSMRTreeAccessor(ILSMHarness lsmHarness, ILSMIndexOperationContext ctx) { } @Override - public ITreeIndexCursor createSearchCursor() { + public ITreeIndexCursor createSearchCursor(boolean exclusive) { return new LSMRTreeSearchCursor(ctx); } diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java index 4e322d447..32f3174e0 100644 --- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java +++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeAbstractCursor.java @@ -163,4 +163,8 @@ public boolean exclusiveLatchNodes() { return false; } + @Override + public void markCurrentTupleAsUpdated() throws HyracksDataException { + throw new HyracksDataException("Updating tuples is not supported with this cursor."); + } } \ No newline at end of file diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java index bfd0260f2..71d83f2aa 100644 --- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java +++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuples.java @@ -167,7 +167,7 @@ public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataExceptio LSMRTreeMemoryComponent flushingComponent = (LSMRTreeMemoryComponent) flushOp.getFlushingComponent(); ITreeIndexAccessor memRTreeAccessor = flushingComponent.getRTree().createAccessor( NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE); - RTreeSearchCursor rtreeScanCursor = (RTreeSearchCursor) memRTreeAccessor.createSearchCursor(); + RTreeSearchCursor rtreeScanCursor = (RTreeSearchCursor) memRTreeAccessor.createSearchCursor(false); SearchPredicate rtreeNullPredicate = new SearchPredicate(null, null); memRTreeAccessor.search(rtreeScanCursor, rtreeNullPredicate); LSMRTreeDiskComponent component = createDiskComponent(componentFactory, flushOp.getRTreeFlushTarget(), null, @@ -177,7 +177,7 @@ public ILSMComponent flush(ILSMIOOperation operation) throws HyracksDataExceptio // scan the memory BTree ITreeIndexAccessor memBTreeAccessor = flushingComponent.getBTree().createAccessor( NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE); - BTreeRangeSearchCursor btreeScanCursor = (BTreeRangeSearchCursor) memBTreeAccessor.createSearchCursor(); + BTreeRangeSearchCursor btreeScanCursor = (BTreeRangeSearchCursor) memBTreeAccessor.createSearchCursor(false); RangePredicate btreeNullPredicate = new RangePredicate(null, null, true, true, null, null); memBTreeAccessor.search(btreeScanCursor, btreeNullPredicate); @@ -298,7 +298,7 @@ public LSMRTreeWithAntiMatterTuplesAccessor(ILSMHarness lsmHarness, ILSMIndexOpe } @Override - public ITreeIndexCursor createSearchCursor() { + public ITreeIndexCursor createSearchCursor(boolean exclusive) { return new LSMRTreeWithAntiMatterTuplesSearchCursor(ctx); } diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java index 0dd228106..aa3fad845 100644 --- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java +++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/LSMRTreeWithAntiMatterTuplesFlushCursor.java @@ -145,20 +145,21 @@ public ICachedPage getPage() { @Override public void setBufferCache(IBufferCache bufferCache) { - // TODO Auto-generated method stub } @Override public void setFileId(int fileId) { - // TODO Auto-generated method stub } @Override public boolean exclusiveLatchNodes() { - // TODO Auto-generated method stub return false; } + @Override + public void markCurrentTupleAsUpdated() throws HyracksDataException { + throw new HyracksDataException("Updating tuples is not supported with this cursor."); + } } diff --git a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java index 03d904397..79465293d 100644 --- a/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java +++ b/hyracks/hyracks-storage-am-lsm-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/impls/TreeTupleSorter.java @@ -222,4 +222,8 @@ public boolean exclusiveLatchNodes() { return false; } + @Override + public void markCurrentTupleAsUpdated() throws HyracksDataException { + throw new HyracksDataException("Updating tuples is not supported with this cursor."); + } } \ No newline at end of file diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java index 7bb358376..266686f0e 100644 --- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java +++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTree.java @@ -201,7 +201,7 @@ private void insert(ITupleReference tuple, IIndexOperationContext ictx) throws H ICachedPage node = ctx.LSNUpdates.get(i); ctx.interiorFrame.setPage(node); ctx.interiorFrame.setPageLsn(incrementGlobalNsn()); - node.releaseWriteLatch(); + node.releaseWriteLatch(true); bufferCache.unpin(node); } } @@ -228,7 +228,7 @@ private ICachedPage findLeaf(RTreeOpContext ctx) throws HyracksDataException { writeLatched = true; if (!ctx.interiorFrame.isLeaf()) { - node.releaseWriteLatch(); + node.releaseWriteLatch(true); writeLatched = false; bufferCache.unpin(node); continue; @@ -246,7 +246,7 @@ private ICachedPage findLeaf(RTreeOpContext ctx) throws HyracksDataException { // Concurrent split detected, go back to parent and // re-choose the best child if (writeLatched) { - node.releaseWriteLatch(); + node.releaseWriteLatch(true); writeLatched = false; bufferCache.unpin(node); } else { @@ -294,7 +294,7 @@ private ICachedPage findLeaf(RTreeOpContext ctx) throws HyracksDataException { // already pointing to the best child ctx.interiorFrame.enlarge(ctx.getTuple(), ctx.cmp); - node.releaseWriteLatch(); + node.releaseWriteLatch(true); writeLatched = false; bufferCache.unpin(node); } else { @@ -303,7 +303,7 @@ private ICachedPage findLeaf(RTreeOpContext ctx) throws HyracksDataException { readLatched = false; bufferCache.unpin(node); } else if (writeLatched) { - node.releaseWriteLatch(); + node.releaseWriteLatch(true); writeLatched = false; bufferCache.unpin(node); } @@ -324,7 +324,7 @@ private ICachedPage findLeaf(RTreeOpContext ctx) throws HyracksDataException { readLatched = false; bufferCache.unpin(node); } else if (writeLatched) { - node.releaseWriteLatch(); + node.releaseWriteLatch(true); writeLatched = false; bufferCache.unpin(node); } @@ -359,7 +359,7 @@ private void insertTuple(ICachedPage node, int pageId, ITupleReference tuple, RT } else if (isLeaf) { // In case of a crash, we un-latch the interior node // inside updateParentForInsert. - node.releaseWriteLatch(); + node.releaseWriteLatch(true); bufferCache.unpin(node); } } @@ -384,7 +384,7 @@ private void insertTuple(ICachedPage node, int pageId, ITupleReference tuple, RT } else if (isLeaf) { // In case of a crash, we un-latch the interior node // inside updateParentForInsert. - node.releaseWriteLatch(); + node.releaseWriteLatch(true); bufferCache.unpin(node); } } @@ -424,12 +424,12 @@ private void insertTuple(ICachedPage node, int pageId, ITupleReference tuple, RT } else if (isLeaf) { // In case of a crash, we un-latch the interior node // inside updateParentForInsert. - node.releaseWriteLatch(); + node.releaseWriteLatch(true); bufferCache.unpin(node); - rightNode.releaseWriteLatch(); + rightNode.releaseWriteLatch(true); bufferCache.unpin(rightNode); } else { - rightNode.releaseWriteLatch(); + rightNode.releaseWriteLatch(true); bufferCache.unpin(rightNode); } @@ -469,16 +469,16 @@ private void insertTuple(ICachedPage node, int pageId, ITupleReference tuple, RT } else if (isLeaf) { // In case of a crash, we un-latch the interior node // inside updateParentForInsert. - node.releaseWriteLatch(); + node.releaseWriteLatch(true); bufferCache.unpin(node); - rightNode.releaseWriteLatch(); + rightNode.releaseWriteLatch(true); bufferCache.unpin(rightNode); - newLeftNode.releaseWriteLatch(); + newLeftNode.releaseWriteLatch(true); bufferCache.unpin(newLeftNode); } else { - rightNode.releaseWriteLatch(); + rightNode.releaseWriteLatch(true); bufferCache.unpin(rightNode); - newLeftNode.releaseWriteLatch(); + newLeftNode.releaseWriteLatch(true); bufferCache.unpin(newLeftNode); } } @@ -508,7 +508,7 @@ private void updateParentForInsert(RTreeOpContext ctx) throws HyracksDataExcepti break; } int rightPage = ctx.interiorFrame.getRightPage(); - parentNode.releaseWriteLatch(); + parentNode.releaseWriteLatch(true); writeLatched = false; bufferCache.unpin(parentNode); @@ -529,7 +529,7 @@ private void updateParentForInsert(RTreeOpContext ctx) throws HyracksDataExcepti ctx.interiorFrame.adjustKey(ctx.splitKey.getLeftTuple(), -1, ctx.cmp); } catch (TreeIndexException e) { if (writeLatched) { - parentNode.releaseWriteLatch(); + parentNode.releaseWriteLatch(true); writeLatched = false; bufferCache.unpin(parentNode); } @@ -544,7 +544,7 @@ private void updateParentForInsert(RTreeOpContext ctx) throws HyracksDataExcepti } finally { if (!succeeded) { if (writeLatched) { - parentNode.releaseWriteLatch(); + parentNode.releaseWriteLatch(true); writeLatched = false; bufferCache.unpin(parentNode); } @@ -634,7 +634,7 @@ private void delete(ITupleReference tuple, RTreeOpContext ctx) throws HyracksDat try { deleteTuple(tupleIndex, ctx); } finally { - ctx.leafFrame.getPage().releaseWriteLatch(); + ctx.leafFrame.getPage().releaseWriteLatch(true); bufferCache.unpin(ctx.leafFrame.getPage()); } } @@ -694,7 +694,7 @@ private int findTupleToDelete(RTreeOpContext ctx) throws HyracksDataException { if (!ctx.leafFrame.isLeaf()) { ctx.pathList.add(pageId, -1, -1); - node.releaseWriteLatch(); + node.releaseWriteLatch(true); writeLatched = false; bufferCache.unpin(node); continue; @@ -707,7 +707,7 @@ private int findTupleToDelete(RTreeOpContext ctx) throws HyracksDataException { if (tupleIndex == -1) { ctx.pathList.add(pageId, parentLsn, -1); - node.releaseWriteLatch(); + node.releaseWriteLatch(true); writeLatched = false; bufferCache.unpin(node); continue; @@ -732,7 +732,7 @@ private int findTupleToDelete(RTreeOpContext ctx) throws HyracksDataException { readLatched = false; bufferCache.unpin(node); } else if (writeLatched) { - node.releaseWriteLatch(); + node.releaseWriteLatch(true); writeLatched = false; bufferCache.unpin(node); } @@ -825,7 +825,7 @@ public void delete(ITupleReference tuple) throws HyracksDataException, TreeIndex } @Override - public ITreeIndexCursor createSearchCursor() { + public ITreeIndexCursor createSearchCursor(boolean exclusive) { return new RTreeSearchCursor((IRTreeInteriorFrame) interiorFrameFactory.createFrame(), (IRTreeLeafFrame) leafFrameFactory.createFrame()); } @@ -907,7 +907,7 @@ public void add(ITupleReference tuple) throws IndexException, HyracksDataExcepti leafFrontier.pageId = freePageManager.getFreePage(metaFrame); - leafFrontier.page.releaseWriteLatch(); + leafFrontier.page.releaseWriteLatch(true); bufferCache.unpin(leafFrontier.page); leafFrontier.page = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, leafFrontier.pageId), @@ -974,7 +974,7 @@ protected void propagateBulk(int level, boolean toRoot) throws HyracksDataExcept propagateBulk(level + 1, toRoot); propagated = true; - frontier.page.releaseWriteLatch(); + frontier.page.releaseWriteLatch(true); bufferCache.unpin(frontier.page); frontier.pageId = freePageManager.getFreePage(metaFrame); diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java index c71b3eb29..ab82f0873 100644 --- a/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java +++ b/hyracks/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/impls/RTreeSearchCursor.java @@ -252,4 +252,9 @@ public void setFileId(int fileId) { public boolean exclusiveLatchNodes() { return false; } + + @Override + public void markCurrentTupleAsUpdated() throws HyracksDataException { + throw new HyracksDataException("Updating tuples is not supported with this cursor."); + } } \ No newline at end of file diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java index 2992dfe47..4162dc27f 100644 --- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java +++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java @@ -474,18 +474,9 @@ public void acquireReadLatch() { latch.readLock().lock(); } - private void acquireWriteLatch(boolean markDirty) { - latch.writeLock().lock(); - if (markDirty) { - if (dirty.compareAndSet(false, true)) { - pinCount.incrementAndGet(); - } - } - } - @Override public void acquireWriteLatch() { - acquireWriteLatch(true); + latch.writeLock().lock(); } @Override @@ -494,7 +485,12 @@ public void releaseReadLatch() { } @Override - public void releaseWriteLatch() { + public void releaseWriteLatch(boolean markDirty) { + if (markDirty) { + if (dirty.compareAndSet(false, true)) { + pinCount.incrementAndGet(); + } + } latch.writeLock().unlock(); } } diff --git a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/ICachedPage.java b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/ICachedPage.java index b5bf3bb23..6c86597fa 100644 --- a/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/ICachedPage.java +++ b/hyracks/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/ICachedPage.java @@ -25,5 +25,5 @@ public interface ICachedPage { public void acquireWriteLatch(); - public void releaseWriteLatch(); + public void releaseWriteLatch(boolean markDirty); } \ No newline at end of file diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java index 02ce7f194..4c158ce6f 100644 --- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java +++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/AbstractSearchOperationCallbackTest.java @@ -88,7 +88,7 @@ private class SearchTask implements Callable { public SearchTask() throws HyracksDataException { this.cb = new SynchronizingSearchOperationCallback(); this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, cb); - this.cursor = accessor.createSearchCursor(); + this.cursor = accessor.createSearchCursor(false); this.predicate = new RangePredicate(); this.builder = new ArrayTupleBuilder(NUM_KEY_FIELDS); this.tuple = new ArrayTupleReference(); diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java index 0a701856a..f814829b8 100644 --- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java +++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexExamplesTest.java @@ -735,7 +735,7 @@ private void orderedScan(IIndexAccessor indexAccessor, ISerializerDeserializer[] if (LOGGER.isLoggable(Level.INFO)) { LOGGER.info("Ordered Scan:"); } - IIndexCursor scanCursor = (IIndexCursor) indexAccessor.createSearchCursor(); + IIndexCursor scanCursor = (IIndexCursor) indexAccessor.createSearchCursor(false); RangePredicate nullPred = new RangePredicate(null, null, true, true, null, null); indexAccessor.search(scanCursor, nullPred); try { @@ -795,7 +795,7 @@ private void rangeSearch(IBinaryComparatorFactory[] cmpFactories, IIndexAccessor String highKeyString = TupleUtils.printTuple(highKey, fieldSerdes); LOGGER.info("Range-Search in: [ " + lowKeyString + ", " + highKeyString + "]"); } - ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(); + ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false); MultiComparator lowKeySearchCmp = BTreeUtils.getSearchMultiComparator(cmpFactories, lowKey); MultiComparator highKeySearchCmp = BTreeUtils.getSearchMultiComparator(cmpFactories, highKey); RangePredicate rangePred = new RangePredicate(lowKey, highKey, true, true, lowKeySearchCmp, highKeySearchCmp); diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexSortedInsertTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexSortedInsertTest.java new file mode 100644 index 000000000..e8b83e879 --- /dev/null +++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexSortedInsertTest.java @@ -0,0 +1,75 @@ +/* + * Copyright 2009-2013 by The Regents of the University of California + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * you may obtain a copy of the License from + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package edu.uci.ics.hyracks.storage.am.btree; + +import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer; +import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference; +import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer; +import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer; +import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType; + +/** + * Tests the BTree insert operation with sorted stream of strings and integer fields using + * various numbers of key and payload fields. + * Each tests first fills a BTree with sorted generated tuples. We compare the + * following operations against expected results: 1. Point searches for all + * tuples. 2. Ordered scan. 3. Disk-order scan. 4. Range search (and prefix + * search for composite keys). + */ +@SuppressWarnings("rawtypes") +public abstract class OrderedIndexSortedInsertTest extends OrderedIndexTestDriver { + + private final OrderedIndexTestUtils orderedIndexTestUtils; + + public OrderedIndexSortedInsertTest(BTreeLeafFrameType[] leafFrameTypesToTest) { + super(leafFrameTypesToTest); + this.orderedIndexTestUtils = new OrderedIndexTestUtils(); + } + + @Override + protected void runTest(ISerializerDeserializer[] fieldSerdes, int numKeys, BTreeLeafFrameType leafType, + ITupleReference lowKey, ITupleReference highKey, ITupleReference prefixLowKey, ITupleReference prefixHighKey) + throws Exception { + OrderedIndexTestContext ctx = createTestContext(fieldSerdes, numKeys, leafType); + ctx.getIndex().create(); + ctx.getIndex().activate(); + // We assume all fieldSerdes are of the same type. Check the first one + // to determine which field types to generate. + if (fieldSerdes[0] instanceof IntegerSerializerDeserializer) { + orderedIndexTestUtils.insertSortedIntTuples(ctx, numTuplesToInsert, getRandom()); + } else if (fieldSerdes[0] instanceof UTF8StringSerializerDeserializer) { + orderedIndexTestUtils.insertSortedStringTuples(ctx, numTuplesToInsert, getRandom()); + } + + orderedIndexTestUtils.checkPointSearches(ctx); + orderedIndexTestUtils.checkScan(ctx); + orderedIndexTestUtils.checkDiskOrderScan(ctx); + + orderedIndexTestUtils.checkRangeSearch(ctx, lowKey, highKey, true, true); + if (prefixLowKey != null && prefixHighKey != null) { + orderedIndexTestUtils.checkRangeSearch(ctx, prefixLowKey, prefixHighKey, true, true); + } + + ctx.getIndex().validate(); + ctx.getIndex().deactivate(); + ctx.getIndex().destroy(); + } + + @Override + protected String getTestOpName() { + return "Insert-Sorted"; + } +} diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java index 9de217bd5..a2a4ba821 100644 --- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java +++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/btree/OrderedIndexTestUtils.java @@ -43,6 +43,7 @@ import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor; import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate; import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor; +import edu.uci.ics.hyracks.storage.am.common.api.IndexException; import edu.uci.ics.hyracks.storage.am.common.exceptions.TreeIndexDuplicateKeyException; import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator; @@ -93,7 +94,7 @@ public void checkRangeSearch(IIndexTestContext ctx, ITupleReference lowKey, ITup } MultiComparator lowKeyCmp = BTreeUtils.getSearchMultiComparator(ctx.getComparatorFactories(), lowKey); MultiComparator highKeyCmp = BTreeUtils.getSearchMultiComparator(ctx.getComparatorFactories(), highKey); - IIndexCursor searchCursor = ctx.getIndexAccessor().createSearchCursor(); + IIndexCursor searchCursor = ctx.getIndexAccessor().createSearchCursor(false); RangePredicate rangePred = new RangePredicate(lowKey, highKey, lowKeyInclusive, highKeyInclusive, lowKeyCmp, highKeyCmp); ctx.getIndexAccessor().search(searchCursor, rangePred); @@ -140,7 +141,7 @@ public void checkPointSearches(IIndexTestContext ictx) throws Exception { LOGGER.info("Testing Point Searches On All Expected Keys."); } OrderedIndexTestContext ctx = (OrderedIndexTestContext) ictx; - IIndexCursor searchCursor = ctx.getIndexAccessor().createSearchCursor(); + IIndexCursor searchCursor = ctx.getIndexAccessor().createSearchCursor(false); ArrayTupleBuilder lowKeyBuilder = new ArrayTupleBuilder(ctx.getKeyFieldCount()); ArrayTupleReference lowKey = new ArrayTupleReference(); @@ -179,6 +180,79 @@ public void checkPointSearches(IIndexTestContext ictx) throws Exception { } } + @SuppressWarnings("unchecked") + public void insertSortedIntTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception { + int fieldCount = ctx.getFieldCount(); + int numKeyFields = ctx.getKeyFieldCount(); + int[] fieldValues = new int[ctx.getFieldCount()]; + int maxValue = (int) Math.ceil(Math.pow(numTuples, 1.0 / (double) numKeyFields)); + Collection tmpCheckTuples = createCheckTuplesCollection(); + for (int i = 0; i < numTuples; i++) { + // Set keys. + setIntKeyFields(fieldValues, numKeyFields, maxValue, rnd); + // Set values. + setIntPayloadFields(fieldValues, numKeyFields, fieldCount); + + // Set expected values. (We also use these as the pre-sorted stream + // for ordered indexes bulk loading). + ctx.insertCheckTuple(createIntCheckTuple(fieldValues, ctx.getKeyFieldCount()), tmpCheckTuples); + } + insertCheckTuples(ctx, tmpCheckTuples); + + // Add tmpCheckTuples to ctx check tuples for comparing searches. + for (CheckTuple checkTuple : tmpCheckTuples) { + ctx.insertCheckTuple(checkTuple, ctx.getCheckTuples()); + } + } + + @SuppressWarnings("unchecked") + public void insertSortedStringTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception { + int fieldCount = ctx.getFieldCount(); + int numKeyFields = ctx.getKeyFieldCount(); + String[] fieldValues = new String[fieldCount]; + TreeSet tmpCheckTuples = new TreeSet(); + for (int i = 0; i < numTuples; i++) { + // Set keys. + for (int j = 0; j < numKeyFields; j++) { + int length = (Math.abs(rnd.nextInt()) % 10) + 1; + fieldValues[j] = getRandomString(length, rnd); + } + // Set values. + for (int j = numKeyFields; j < fieldCount; j++) { + fieldValues[j] = getRandomString(5, rnd); + } + // Set expected values. We also use these as the pre-sorted stream + // for bulk loading. + ctx.insertCheckTuple(createStringCheckTuple(fieldValues, ctx.getKeyFieldCount()), tmpCheckTuples); + } + insertCheckTuples(ctx, tmpCheckTuples); + + // Add tmpCheckTuples to ctx check tuples for comparing searches. + for (CheckTuple checkTuple : tmpCheckTuples) { + ctx.insertCheckTuple(checkTuple, ctx.getCheckTuples()); + } + } + + public static void insertCheckTuples(IIndexTestContext ctx, Collection checkTuples) + throws HyracksDataException, IndexException { + int fieldCount = ctx.getFieldCount(); + int numTuples = checkTuples.size(); + ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(fieldCount); + ArrayTupleReference tuple = new ArrayTupleReference(); + + int c = 1; + for (CheckTuple checkTuple : checkTuples) { + if (LOGGER.isLoggable(Level.INFO)) { + if (c % (numTuples / 10) == 0) { + LOGGER.info("Inserting Tuple " + c + "/" + numTuples); + } + } + createTupleFromCheckTuple(checkTuple, tupleBuilder, tuple, ctx.getFieldSerdes()); + ctx.getIndexAccessor().insert(tuple); + c++; + } + } + @SuppressWarnings("unchecked") public void insertStringTuples(IIndexTestContext ctx, int numTuples, Random rnd) throws Exception { int fieldCount = ctx.getFieldCount(); diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java index b5742b5ca..291980514 100644 --- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java +++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/common/TreeIndexTestUtils.java @@ -98,7 +98,7 @@ public void checkScan(IIndexTestContext ctx) throws Exception { if (LOGGER.isLoggable(Level.INFO)) { LOGGER.info("Testing Scan."); } - ITreeIndexCursor scanCursor = (ITreeIndexCursor) ctx.getIndexAccessor().createSearchCursor(); + ITreeIndexCursor scanCursor = (ITreeIndexCursor) ctx.getIndexAccessor().createSearchCursor(false); ISearchPredicate nullPred = createNullSearchPredicate(); ctx.getIndexAccessor().search(scanCursor, nullPred); Iterator checkIter = ctx.getCheckTuples().iterator(); diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java index 705a976bc..6a519d006 100644 --- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java +++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/AbstractRTreeExamplesTest.java @@ -727,7 +727,7 @@ private void scan(IIndexAccessor indexAccessor, ISerializerDeserializer[] fieldS if (LOGGER.isLoggable(Level.INFO)) { LOGGER.info("Scan:"); } - ITreeIndexCursor scanCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(); + ITreeIndexCursor scanCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false); SearchPredicate nullPred = new SearchPredicate(null, null); indexAccessor.search(scanCursor, nullPred); try { @@ -786,7 +786,7 @@ private void rangeSearch(IBinaryComparatorFactory[] cmpFactories, IIndexAccessor String kString = TupleUtils.printTuple(key, fieldSerdes); LOGGER.info("Range-Search using key: " + kString); } - ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(); + ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor(false); MultiComparator cmp = RTreeUtils.getSearchMultiComparator(cmpFactories, key); SearchPredicate rangePred = new SearchPredicate(key, cmp); indexAccessor.search(rangeCursor, rangePred); diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTestUtils.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTestUtils.java index f93a00774..af7cf145d 100644 --- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTestUtils.java +++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/RTreeTestUtils.java @@ -67,7 +67,7 @@ public void checkRangeSearch(IIndexTestContext ictx, ITupleReference key) throws AbstractRTreeTestContext ctx = (AbstractRTreeTestContext) ictx; MultiComparator cmp = RTreeUtils.getSearchMultiComparator(ctx.getComparatorFactories(), key); - ITreeIndexCursor searchCursor = (ITreeIndexCursor) ctx.getIndexAccessor().createSearchCursor(); + ITreeIndexCursor searchCursor = (ITreeIndexCursor) ctx.getIndexAccessor().createSearchCursor(false); SearchPredicate searchPred = new SearchPredicate(key, cmp); ctx.getIndexAccessor().search(searchCursor, searchPred); diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeSortedInsertTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeSortedInsertTest.java new file mode 100644 index 000000000..5e24c5287 --- /dev/null +++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/BTreeSortedInsertTest.java @@ -0,0 +1,68 @@ +/* + * Copyright 2009-2013 by The Regents of the University of California + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * you may obtain a copy of the License from + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package edu.uci.ics.hyracks.storage.am.btree; + +import java.util.Random; + +import org.junit.After; +import org.junit.Before; + +import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer; +import edu.uci.ics.hyracks.api.exceptions.HyracksDataException; +import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeLeafFrameType; +import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestContext; +import edu.uci.ics.hyracks.storage.am.btree.util.BTreeTestHarness; + +/** + * Tests the BTree insert operation with sorted stream of strings and integer fields using + * various numbers of key and payload fields. Each tests first fills a BTree with + * randomly generated tuples. We compare the following operations against expected results: + * 1) Point searches for all tuples + * 2) Ordered scan + * 3) Disk-order scan + * 4) Range search (and prefix search for composite keys) + */ +public class BTreeSortedInsertTest extends OrderedIndexSortedInsertTest { + + private final BTreeTestHarness harness = new BTreeTestHarness(); + + public BTreeSortedInsertTest() { + super(BTreeTestHarness.LEAF_FRAMES_TO_TEST); + } + + @Before + public void setUp() throws HyracksDataException { + harness.setUp(); + } + + @After + public void tearDown() throws HyracksDataException { + harness.tearDown(); + } + + @SuppressWarnings("rawtypes") + @Override + protected OrderedIndexTestContext createTestContext(ISerializerDeserializer[] fieldSerdes, int numKeys, + BTreeLeafFrameType leafType) throws Exception { + return BTreeTestContext.create(harness.getBufferCache(), harness.getFileMapProvider(), + harness.getFileReference(), fieldSerdes, numKeys, leafType); + } + + @Override + protected Random getRandom() { + return harness.getRandom(); + } +} diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java index 29da625c9..5f77d689f 100644 --- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java +++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/StorageManagerTest.java @@ -154,7 +154,7 @@ private void unpinRandomPage() { if (LOGGER.isLoggable(Level.INFO)) { LOGGER.info(workerId + " X UNLATCHING: " + plPage.pageId); } - plPage.page.releaseWriteLatch(); + plPage.page.releaseWriteLatch(true); } } if (LOGGER.isLoggable(Level.INFO)) { diff --git a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java index 22d3e6ade..2d7a3189e 100644 --- a/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java +++ b/hyracks/hyracks-tests/hyracks-storage-am-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/btree/multithread/BTreeTestWorker.java @@ -51,7 +51,7 @@ public BTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, @Override public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, IndexException { BTree.BTreeAccessor accessor = (BTree.BTreeAccessor) indexAccessor; - ITreeIndexCursor searchCursor = accessor.createSearchCursor(); + ITreeIndexCursor searchCursor = accessor.createSearchCursor(false); ITreeIndexCursor diskOrderScanCursor = accessor.createDiskOrderScanCursor(); MultiComparator cmp = accessor.getOpContext().cmp; RangePredicate rangePred = new RangePredicate(tuple, tuple, true, true, cmp, cmp); diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java index f1a535536..afaf30b8e 100644 --- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java +++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/LSMBTreeSearchOperationCallbackTest.java @@ -97,7 +97,7 @@ private class SearchTask implements Callable { public SearchTask() throws HyracksDataException { this.cb = new SynchronizingSearchOperationCallback(); this.accessor = index.createAccessor(NoOpOperationCallback.INSTANCE, cb); - this.cursor = accessor.createSearchCursor(); + this.cursor = accessor.createSearchCursor(false); this.predicate = new RangePredicate(); this.builder = new ArrayTupleBuilder(NUM_KEY_FIELDS); this.tuple = new ArrayTupleReference(); diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java index 35ecc20ea..8ab5d2259 100644 --- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java +++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/btree/multithread/LSMBTreeTestWorker.java @@ -52,7 +52,7 @@ public LSMBTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelecto @Override public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, IndexException { LSMBTreeAccessor accessor = (LSMBTreeAccessor) indexAccessor; - IIndexCursor searchCursor = accessor.createSearchCursor(); + IIndexCursor searchCursor = accessor.createSearchCursor(false); MultiComparator cmp = accessor.getMultiComparator(); RangePredicate rangePred = new RangePredicate(tuple, tuple, true, true, cmp, cmp); diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java index 35afd099c..34b3ff752 100644 --- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java +++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/multithread/LSMInvertedIndexTestWorker.java @@ -58,7 +58,7 @@ public LSMInvertedIndexTestWorker(DataGenThread dataGen, TestOperationSelector o @Override public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, IndexException { LSMInvertedIndexAccessor accessor = (LSMInvertedIndexAccessor) indexAccessor; - IIndexCursor searchCursor = accessor.createSearchCursor(); + IIndexCursor searchCursor = accessor.createSearchCursor(false); IIndexCursor rangeSearchCursor = accessor.createRangeSearchCursor(); RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null); IBinaryTokenizerFactory tokenizerFactory = invIndex.getTokenizerFactory(); diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java index c4e1ac525..58e0aceda 100644 --- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java +++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/invertedindex/util/LSMInvertedIndexTestUtils.java @@ -510,7 +510,7 @@ public static void testIndexSearch(LSMInvertedIndexTestContext testCtx, TupleGen searchPred.setQueryTuple(searchDocument); searchPred.setQueryFieldIndex(0); - IIndexCursor resultCursor = accessor.createSearchCursor(); + IIndexCursor resultCursor = accessor.createSearchCursor(false); boolean panic = false; try { accessor.search(resultCursor, searchPred); diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java index c7a86d962..b52b518a3 100644 --- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java +++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeTestWorker.java @@ -50,7 +50,7 @@ public LSMRTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelecto @Override public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, IndexException { LSMRTreeAccessor accessor = (LSMRTreeAccessor) indexAccessor; - ITreeIndexCursor searchCursor = accessor.createSearchCursor(); + ITreeIndexCursor searchCursor = accessor.createSearchCursor(false); MultiComparator cmp = accessor.getMultiComparator(); SearchPredicate rangePred = new SearchPredicate(tuple, cmp); diff --git a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java index 383cbc42a..eef80d695 100644 --- a/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java +++ b/hyracks/hyracks-tests/hyracks-storage-am-lsm-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/lsm/rtree/multithread/LSMRTreeWithAntiMatterTuplesTestWorker.java @@ -39,7 +39,7 @@ public LSMRTreeWithAntiMatterTuplesTestWorker(DataGenThread dataGen, TestOperati @Override public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, IndexException { LSMRTreeWithAntiMatterTuplesAccessor accessor = (LSMRTreeWithAntiMatterTuplesAccessor) indexAccessor; - ITreeIndexCursor searchCursor = accessor.createSearchCursor(); + ITreeIndexCursor searchCursor = accessor.createSearchCursor(false); MultiComparator cmp = accessor.getMultiComparator(); SearchPredicate rangePred = new SearchPredicate(tuple, cmp); diff --git a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java index bc5d33120..04cf044b6 100644 --- a/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java +++ b/hyracks/hyracks-tests/hyracks-storage-am-rtree-test/src/test/java/edu/uci/ics/hyracks/storage/am/rtree/multithread/RTreeTestWorker.java @@ -49,7 +49,7 @@ public RTreeTestWorker(DataGenThread dataGen, TestOperationSelector opSelector, @Override public void performOp(ITupleReference tuple, TestOperation op) throws HyracksDataException, IndexException { RTree.RTreeAccessor accessor = (RTree.RTreeAccessor) indexAccessor; - IIndexCursor searchCursor = accessor.createSearchCursor(); + IIndexCursor searchCursor = accessor.createSearchCursor(false); ITreeIndexCursor diskOrderScanCursor = accessor.createDiskOrderScanCursor(); MultiComparator cmp = accessor.getOpContext().cmp; SearchPredicate rangePred = new SearchPredicate(tuple, cmp); diff --git a/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTest.java b/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTest.java index 577abbbf6..ba1715f9e 100644 --- a/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTest.java +++ b/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheRegressionTest.java @@ -81,7 +81,7 @@ private void flushBehaviorTest(boolean deleteFile) throws IOException { buf.put(Byte.MAX_VALUE); } } finally { - writePage.releaseWriteLatch(); + writePage.releaseWriteLatch(true); bufferCache.unpin(writePage); } bufferCache.closeFile(firstFileId); diff --git a/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheTest.java b/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheTest.java index c67386eac..118a27c5f 100644 --- a/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheTest.java +++ b/hyracks/hyracks-tests/hyracks-storage-common-test/src/test/java/edu/uci/ics/hyracks/storage/common/BufferCacheTest.java @@ -91,7 +91,7 @@ public void simpleOpenPinCloseTest() throws HyracksDataException { bufferCache.unpin(page2); } finally { - page.releaseWriteLatch(); + page.releaseWriteLatch(true); bufferCache.unpin(page); } @@ -228,7 +228,7 @@ public void contentCheckingMaxOpenFilesTest() throws HyracksDataException { } pageContents.put(fileId, values); } finally { - page.releaseWriteLatch(); + page.releaseWriteLatch(true); bufferCache.unpin(page); } } diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java index 26cb8d0b3..a42b411d8 100644 --- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java +++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/graph/Vertex.java @@ -258,8 +258,9 @@ public final boolean isHalted() { @Override final public void readFields(DataInput in) throws IOException { reset(); - if (vertexId == null) + if (vertexId == null) { vertexId = BspUtils. createVertexIndex(getContext().getConfiguration()); + } vertexId.readFields(in); delegate.setVertexId(vertexId); boolean hasVertexValue = in.readBoolean(); @@ -576,15 +577,6 @@ public static final TaskAttemptContext getContext() { return context; } - /** - * Pregelix internal use only - * - * @param context - */ - public static final void setContext(TaskAttemptContext context) { - Vertex.context = context; - } - @Override public int hashCode() { return vertexId.hashCode(); diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java index 6549c524b..075de03a4 100644 --- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java +++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/job/PregelixJob.java @@ -20,13 +20,13 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.Job; -import edu.uci.ics.pregelix.api.graph.GlobalAggregator; import edu.uci.ics.pregelix.api.graph.MessageCombiner; import edu.uci.ics.pregelix.api.graph.NormalizedKeyComputer; import edu.uci.ics.pregelix.api.graph.Vertex; import edu.uci.ics.pregelix.api.graph.VertexPartitioner; import edu.uci.ics.pregelix.api.io.VertexInputFormat; import edu.uci.ics.pregelix.api.io.VertexOutputFormat; +import edu.uci.ics.pregelix.api.util.GlobalCountAggregator; /** * This class represents a Pregelix job. @@ -80,6 +80,8 @@ public class PregelixJob extends Job { public static final String RECOVERY_COUNT = "pregelix.recoveryCount"; /** the checkpoint interval */ public static final String CKP_INTERVAL = "pregelix.ckpinterval"; + /** comma */ + public static final String COMMA_STR = ","; /** * Construct a Pregelix job from an existing configuration @@ -89,6 +91,7 @@ public class PregelixJob extends Job { */ public PregelixJob(Configuration conf) throws IOException { super(conf); + this.addGlobalAggregatorClass(GlobalCountAggregator.class); } /** @@ -100,6 +103,7 @@ public PregelixJob(Configuration conf) throws IOException { */ public PregelixJob(String jobName) throws IOException { super(new Configuration(), jobName); + this.addGlobalAggregatorClass(GlobalCountAggregator.class); } /** @@ -113,6 +117,7 @@ public PregelixJob(String jobName) throws IOException { */ public PregelixJob(Configuration conf, String jobName) throws IOException { super(conf, jobName); + this.addGlobalAggregatorClass(GlobalCountAggregator.class); } /** @@ -161,8 +166,10 @@ final public void setMessageCombinerClass(Class vertexCombinerClass) { * @param globalAggregatorClass * Determines how messages are globally aggregated */ - final public void setGlobalAggregatorClass(Class globalAggregatorClass) { - getConfiguration().setClass(GLOBAL_AGGREGATOR_CLASS, globalAggregatorClass, GlobalAggregator.class); + final public void addGlobalAggregatorClass(Class globalAggregatorClass) { + String aggStr = globalAggregatorClass.getName(); + String classes = getConfiguration().get(GLOBAL_AGGREGATOR_CLASS); + conf.set(GLOBAL_AGGREGATOR_CLASS, classes == null ? aggStr : classes + COMMA_STR + aggStr); } /** @@ -181,6 +188,15 @@ final public void setDynamicVertexValueSize(boolean incStateLengthDynamically) { getConfiguration().setBoolean(INCREASE_STATE_LENGTH, incStateLengthDynamically); } + /** + * Set whether the vertex state length is fixed + * + * @param jobId + */ + final public void setFixedVertexValueSize(boolean fixedSize) { + getConfiguration().setBoolean(INCREASE_STATE_LENGTH, !fixedSize); + } + /** * Set the frame size for a job * diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java index 4ee1deb4e..a5d9cd74e 100644 --- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java +++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/BspUtils.java @@ -15,6 +15,9 @@ package edu.uci.ics.pregelix.api.util; +import java.util.ArrayList; +import java.util.List; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableComparable; @@ -37,11 +40,6 @@ * them. */ public class BspUtils { - private static Configuration defaultConf = null; - - public static void setDefaultConfiguration(Configuration conf) { - defaultConf = conf; - } /** * Get the user's subclassed {@link VertexInputFormat}. @@ -122,10 +120,19 @@ public static Class> getGlobalAggregatorClass( + public static List>> getGlobalAggregatorClasses( Configuration conf) { - return (Class>) conf.getClass(PregelixJob.GLOBAL_AGGREGATOR_CLASS, - GlobalCountAggregator.class, GlobalAggregator.class); + String aggStrs = conf.get(PregelixJob.GLOBAL_AGGREGATOR_CLASS); + String[] classnames = aggStrs.split(PregelixJob.COMMA_STR); + try { + List>> classes = new ArrayList>>(); + for (int i = 0; i < classnames.length; i++) { + classes.add((Class>) conf.getClassByName(classnames[i])); + } + return classes; + } catch (ClassNotFoundException e) { + throw new RuntimeException(e); + } } public static String getJobId(Configuration conf) { @@ -166,10 +173,52 @@ public static NormalizedKeyComputer createNormalizedKeyComputer(Configuration co * @return Instantiated user vertex combiner class */ @SuppressWarnings("rawtypes") - public static GlobalAggregator createGlobalAggregator( + public static List createGlobalAggregators( Configuration conf) { - Class> globalAggregatorClass = getGlobalAggregatorClass(conf); - return ReflectionUtils.newInstance(globalAggregatorClass, conf); + List>> globalAggregatorClasses = getGlobalAggregatorClasses(conf); + List aggs = new ArrayList(); + for (Class> globalAggClass : globalAggregatorClasses) { + aggs.add(ReflectionUtils.newInstance(globalAggClass, conf)); + } + return aggs; + } + + /** + * Get global aggregator class names + * + * @param conf + * Configuration to check + * @return An array of Global aggregator names + */ + @SuppressWarnings("rawtypes") + public static String[] getGlobalAggregatorClassNames( + Configuration conf) { + List>> globalAggregatorClasses = getGlobalAggregatorClasses(conf); + String[] aggClassNames = new String[globalAggregatorClasses.size()]; + int i = 0; + for (Class> globalAggClass : globalAggregatorClasses) { + aggClassNames[i++] = globalAggClass.getName(); + } + return aggClassNames; + } + + /** + * Get global aggregator class names + * + * @param conf + * Configuration to check + * @return An array of Global aggregator names + */ + @SuppressWarnings("rawtypes") + public static String[] getPartialAggregateValueClassNames( + Configuration conf) { + String[] gloablAggClassNames = getGlobalAggregatorClassNames(conf); + String[] partialAggValueClassNames = new String[gloablAggClassNames.length]; + int i = 0; + for (String globalAggClassName : gloablAggClassNames) { + partialAggValueClassNames[i++] = getPartialAggregateValueClass(conf, globalAggClassName).getName(); + } + return partialAggValueClassNames; } /** @@ -209,8 +258,6 @@ public static Class getVertexIndexClass(Configuration conf) { - if (conf == null) - conf = defaultConf; return (Class) conf.getClass(PregelixJob.VERTEX_INDEX_CLASS, WritableComparable.class); } @@ -302,8 +349,6 @@ public static E createEdgeValue(Configuration conf) { */ @SuppressWarnings("unchecked") public static Class getMessageValueClass(Configuration conf) { - if (conf == null) - conf = defaultConf; return (Class) conf.getClass(PregelixJob.MESSAGE_VALUE_CLASS, Writable.class); } @@ -315,10 +360,8 @@ public static Class getMessageValueClass(Configur * @return User's global aggregate value class */ @SuppressWarnings("unchecked") - public static Class getPartialAggregateValueClass(Configuration conf) { - if (conf == null) - conf = defaultConf; - return (Class) conf.getClass(PregelixJob.PARTIAL_AGGREGATE_VALUE_CLASS, Writable.class); + public static Class getPartialAggregateValueClass(Configuration conf, String aggClassName) { + return (Class) conf.getClass(PregelixJob.PARTIAL_AGGREGATE_VALUE_CLASS + "$" + aggClassName, Writable.class); } /** @@ -330,8 +373,6 @@ public static Class getPartialAggregateValueClass(Config */ @SuppressWarnings("unchecked") public static Class getPartialCombineValueClass(Configuration conf) { - if (conf == null) - conf = defaultConf; return (Class) conf.getClass(PregelixJob.PARTIAL_COMBINE_VALUE_CLASS, Writable.class); } @@ -344,8 +385,6 @@ public static Class getPartialCombineValueClass(Configur */ @SuppressWarnings("unchecked") public static Class getNormalizedKeyComputerClass(Configuration conf) { - if (conf == null) - conf = defaultConf; return (Class) conf.getClass(PregelixJob.NMK_COMPUTER_CLASS, NormalizedKeyComputer.class); } @@ -358,10 +397,8 @@ public static Class getNormalizedKeyComputerCla * @return User's global aggregate value class */ @SuppressWarnings("unchecked") - public static Class getFinalAggregateValueClass(Configuration conf) { - if (conf == null) - conf = defaultConf; - return (Class) conf.getClass(PregelixJob.FINAL_AGGREGATE_VALUE_CLASS, Writable.class); + public static Class getFinalAggregateValueClass(Configuration conf, String aggClassName) { + return (Class) conf.getClass(PregelixJob.FINAL_AGGREGATE_VALUE_CLASS + "$" + aggClassName, Writable.class); } /** @@ -389,8 +426,8 @@ public static M createMessageValue(Configuration con * Configuration to check * @return Instantiated user aggregate value */ - public static M createPartialAggregateValue(Configuration conf) { - Class aggregateValueClass = getPartialAggregateValueClass(conf); + public static M createPartialAggregateValue(Configuration conf, String aggClassName) { + Class aggregateValueClass = getPartialAggregateValueClass(conf, aggClassName); try { return aggregateValueClass.newInstance(); } catch (InstantiationException e) { @@ -400,6 +437,29 @@ public static M createPartialAggregateValue(Configuration c } } + /** + * Create the list of user partial aggregate values + * + * @param conf + * Configuration to check + * @return Instantiated user partial aggregate values + */ + public static List createPartialAggregateValues(Configuration conf) { + String[] aggClassNames = BspUtils.getGlobalAggregatorClassNames(conf); + List aggValueList = new ArrayList(); + for (String aggClassName : aggClassNames) { + Class aggregateValueClass = getPartialAggregateValueClass(conf, aggClassName); + try { + aggValueList.add(aggregateValueClass.newInstance()); + } catch (InstantiationException e) { + throw new IllegalArgumentException("createAggregateValue: Failed to instantiate", e); + } catch (IllegalAccessException e) { + throw new IllegalArgumentException("createAggregateValue: Illegally accessed", e); + } + } + return aggValueList; + } + /** * Create a user partial combine value * @@ -431,8 +491,8 @@ public static M createPartialCombineValue(Configuration con * Configuration to check * @return Instantiated user aggregate value */ - public static M createFinalAggregateValue(Configuration conf) { - Class aggregateValueClass = getFinalAggregateValueClass(conf); + public static M createFinalAggregateValue(Configuration conf, String aggClassName) { + Class aggregateValueClass = getFinalAggregateValueClass(conf, aggClassName); try { return aggregateValueClass.newInstance(); } catch (InstantiationException e) { @@ -442,6 +502,29 @@ public static M createFinalAggregateValue(Configuration con } } + /** + * Create the list of user aggregate values + * + * @param conf + * Configuration to check + * @return Instantiated user aggregate value + */ + public static List createFinalAggregateValues(Configuration conf) { + String[] aggClassNames = BspUtils.getGlobalAggregatorClassNames(conf); + List aggValueList = new ArrayList(); + for (String aggClassName : aggClassNames) { + Class aggregateValueClass = getFinalAggregateValueClass(conf, aggClassName); + try { + aggValueList.add(aggregateValueClass.newInstance()); + } catch (InstantiationException e) { + throw new IllegalArgumentException("createAggregateValue: Failed to instantiate", e); + } catch (IllegalAccessException e) { + throw new IllegalArgumentException("createAggregateValue: Illegally accessed", e); + } + } + return aggValueList; + } + /** * Create a user aggregate value * @@ -488,8 +571,6 @@ public static ICheckpointHook createCheckpointHook(Configuration conf) { */ @SuppressWarnings({ "unchecked", "rawtypes" }) public static Class getVertexPartitionerClass(Configuration conf) { - if (conf == null) - conf = defaultConf; return (Class) conf.getClass(PregelixJob.PARTITIONER_CLASS, null, VertexPartitioner.class); } @@ -502,8 +583,6 @@ public static Class getVertexPartitionerClass(C */ @SuppressWarnings("unchecked") public static Class getCheckpointHookClass(Configuration conf) { - if (conf == null) - conf = defaultConf; return (Class) conf.getClass(PregelixJob.CKP_CLASS, DefaultCheckpointHook.class, ICheckpointHook.class); } @@ -515,7 +594,7 @@ public static Class getCheckpointHookClass(Config * @return the boolean setting of the parameter, by default it is false */ public static boolean getDynamicVertexValueSize(Configuration conf) { - return conf.getBoolean(PregelixJob.INCREASE_STATE_LENGTH, false); + return conf.getBoolean(PregelixJob.INCREASE_STATE_LENGTH, true); } /** diff --git a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/FrameTupleUtils.java b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/FrameTupleUtils.java index a0f67e393..922920ea6 100644 --- a/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/FrameTupleUtils.java +++ b/pregelix/pregelix-api/src/main/java/edu/uci/ics/pregelix/api/util/FrameTupleUtils.java @@ -52,10 +52,12 @@ public static void flushTuplesFinal(FrameTupleAppender appender, IFrameWriter wr public static void flushTupleToHDFS(ArrayTupleBuilder atb, Configuration conf, long superStep) throws HyracksDataException { try { - if (atb.getSize()>0) { + if (atb.getSize() > 0) { FileSystem dfs = FileSystem.get(conf); - String fileName = BspUtils.getGlobalAggregateSpillingDirName(conf, superStep) +"/" + UUID.randomUUID(); - FSDataOutputStream dos = dfs.create(new Path(fileName), true); + String fileName = BspUtils.getGlobalAggregateSpillingDirName(conf, superStep) + "/" + UUID.randomUUID(); + Path path = new Path(fileName); + FSDataOutputStream dos = dfs.create(path, true); + // write the partial aggregate value dos.write(atb.getByteArray(), 0, atb.getSize()); dos.flush(); dos.close(); diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java index c0825dde0..f703fcc42 100644 --- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java +++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGen.java @@ -131,7 +131,7 @@ public abstract class JobGen implements IJobGen { protected PregelixJob pregelixJob; protected IIndexLifecycleManagerProvider lcManagerProvider = IndexLifeCycleManagerProvider.INSTANCE; protected IStorageManagerInterface storageManagerInterface = StorageManagerInterface.INSTANCE; - protected String jobId = new UUID(System.currentTimeMillis(), System.nanoTime()).toString(); + protected String jobId = UUID.randomUUID().toString(); protected int frameSize = ClusterConfig.getFrameSize(); protected int maxFrameNumber = (int) (((long) 32 * MB) / frameSize); @@ -181,14 +181,18 @@ private void initJobConfiguration() { conf.setClass(PregelixJob.EDGE_VALUE_CLASS, (Class) edgeValueType, Writable.class); conf.setClass(PregelixJob.MESSAGE_VALUE_CLASS, (Class) messageValueType, Writable.class); - Class aggregatorClass = BspUtils.getGlobalAggregatorClass(conf); - if (!aggregatorClass.equals(GlobalAggregator.class)) { - List argTypes = ReflectionUtils.getTypeArguments(GlobalAggregator.class, aggregatorClass); - Type partialAggregateValueType = argTypes.get(4); - conf.setClass(PregelixJob.PARTIAL_AGGREGATE_VALUE_CLASS, (Class) partialAggregateValueType, - Writable.class); - Type finalAggregateValueType = argTypes.get(5); - conf.setClass(PregelixJob.FINAL_AGGREGATE_VALUE_CLASS, (Class) finalAggregateValueType, Writable.class); + List aggregatorClasses = BspUtils.getGlobalAggregatorClasses(conf); + for (int i = 0; i < aggregatorClasses.size(); i++) { + Class aggregatorClass = (Class) aggregatorClasses.get(i); + if (!aggregatorClass.equals(GlobalAggregator.class)) { + List argTypes = ReflectionUtils.getTypeArguments(GlobalAggregator.class, aggregatorClass); + Type partialAggregateValueType = argTypes.get(4); + conf.setClass(PregelixJob.PARTIAL_AGGREGATE_VALUE_CLASS + "$" + aggregatorClass.getName(), + (Class) partialAggregateValueType, Writable.class); + Type finalAggregateValueType = argTypes.get(5); + conf.setClass(PregelixJob.FINAL_AGGREGATE_VALUE_CLASS + "$" + aggregatorClass.getName(), + (Class) finalAggregateValueType, Writable.class); + } } Class combinerClass = BspUtils.getMessageCombinerClass(conf); @@ -254,7 +258,7 @@ public JobSpecification scanSortPrintGraph(String nodeName, String path) throws } catch (Exception e) { throw new HyracksDataException(e); } - RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), vertexClass.getName()); IConfigurationFactory confFactory = new ConfigurationFactory(conf); String[] readSchedule = ClusterConfig.getHdfsScheduler().getLocationConstraints(splits); @@ -283,7 +287,7 @@ public JobSpecification scanSortPrintGraph(String nodeName, String path) throws IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results); IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(confFactory); IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( - vertexIdClass.getName(), vertexClass.getName()); + conf, vertexIdClass.getName(), vertexClass.getName()); VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, inputRdFactory, resultFileSplitProvider, preHookFactory, null); PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { "nc1" }); @@ -325,7 +329,7 @@ public JobSpecification scanIndexPrintGraph(String nodeName, String path) throws * construct btree search operator */ IConfigurationFactory confFactory = new ConfigurationFactory(conf); - RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), vertexClass.getName()); IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1]; comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(0, vertexIdClass);; @@ -347,7 +351,7 @@ public JobSpecification scanIndexPrintGraph(String nodeName, String path) throws IFileSplitProvider resultFileSplitProvider = new ConstantFileSplitProvider(results); IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(confFactory); IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( - vertexIdClass.getName(), vertexClass.getName()); + conf, vertexIdClass.getName(), vertexClass.getName()); VertexWriteOperatorDescriptor writer = new VertexWriteOperatorDescriptor(spec, inputRdFactory, resultFileSplitProvider, preHookFactory, null); PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, writer, new String[] { "nc1" }); @@ -478,7 +482,7 @@ protected ITuplePartitionComputerFactory getVertexPartitionComputerFactory() { return new VertexPartitionComputerFactory(confFactory); } else { return new VertexIdPartitionComputerFactory(new WritableSerializerDeserializerFactory( - BspUtils.getVertexIndexClass(conf))); + BspUtils.getVertexIndexClass(conf)), confFactory); } } @@ -514,7 +518,7 @@ private JobSpecification loadHDFSData(PregelixJob job) throws HyracksException, } catch (Exception e) { throw new HyracksDataException(e); } - RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), vertexClass.getName()); IConfigurationFactory confFactory = new ConfigurationFactory(conf); String[] readSchedule = ClusterConfig.getHdfsScheduler().getLocationConstraints(splits); @@ -585,7 +589,7 @@ private JobSpecification scanIndexWriteToHDFS(Configuration conf, boolean ckpoin * construct btree search operator */ IConfigurationFactory confFactory = new ConfigurationFactory(conf); - RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), vertexClass.getName()); IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1]; comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(0, vertexIdClass);; @@ -614,10 +618,11 @@ private JobSpecification scanIndexWriteToHDFS(Configuration conf, boolean ckpoin /** * construct write file operator */ + IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(confFactory); IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( - vertexIdClass.getName(), vertexClass.getName()); + conf, vertexIdClass.getName(), vertexClass.getName()); VertexFileWriteOperatorDescriptor writer = new VertexFileWriteOperatorDescriptor(spec, confFactory, - inputRdFactory); + inputRdFactory, preHookFactory); ClusterConfig.setLocationConstraint(spec, writer); /** @@ -656,7 +661,7 @@ protected JobSpecification[] generateStateCheckpointing(int lastSuccessfulIterat /** * source aggregate */ - RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(), + RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), MsgList.class.getName()); /** @@ -669,7 +674,7 @@ protected JobSpecification[] generateStateCheckpointing(int lastSuccessfulIterat * construct the materializing write operator */ MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal, - false); + false, jobId, lastSuccessfulIteration + 1); ClusterConfig.setLocationConstraint(spec, materializeRead); String checkpointPath = BspUtils.getMessageCheckpointPath(conf, lastSuccessfulIteration);; @@ -680,7 +685,7 @@ protected JobSpecification[] generateStateCheckpointing(int lastSuccessfulIterat tmpJob.setOutputValueClass(MsgList.class); IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( - vertexIdClass.getName(), MsgList.class.getName()); + conf, vertexIdClass.getName(), MsgList.class.getName()); HDFSFileWriteOperatorDescriptor hdfsWriter = new HDFSFileWriteOperatorDescriptor(spec, tmpJob, inputRdFactory); ClusterConfig.setLocationConstraint(spec, hdfsWriter); @@ -720,7 +725,7 @@ protected JobSpecification[] generateStateCheckpointLoading(int lastCheckpointed } catch (Exception e) { throw new HyracksDataException(e); } - RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), MsgList.class.getName()); String[] readSchedule = ClusterConfig.getHdfsScheduler().getLocationConstraints(splits); HDFSReadOperatorDescriptor scanner = new HDFSReadOperatorDescriptor(spec, recordDescriptor, tmpJob, splits, @@ -740,7 +745,7 @@ protected JobSpecification[] generateStateCheckpointLoading(int lastCheckpointed * construct the materializing write operator */ MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, - recordDescriptor); + recordDescriptor, jobId, lastCheckpointedIteration); ClusterConfig.setLocationConstraint(spec, materialize); /** construct runtime hook */ diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java index 120855653..1bad40174 100644 --- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java +++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenInnerJoin.java @@ -101,7 +101,7 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE Class> vertexIdClass = BspUtils.getVertexIndexClass(conf); Class vertexClass = BspUtils.getVertexClass(conf); Class messageValueClass = BspUtils.getMessageValueClass(conf); - Class partialAggregateValueClass = BspUtils.getPartialAggregateValueClass(conf); + String[] partialAggregateValueClassNames = BspUtils.getPartialAggregateValueClassNames(conf); IConfigurationFactory confFactory = new ConfigurationFactory(conf); JobSpecification spec = new JobSpecification(); @@ -125,7 +125,7 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE /** * construct btree search and function call update operator */ - RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), vertexClass.getName()); IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1]; comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass); @@ -134,20 +134,21 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE typeTraits[0] = new TypeTraits(false); typeTraits[1] = new TypeTraits(false); - RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(VLongWritable.class.getName()); - RecordDescriptor rdPartialAggregate = DataflowUtils - .getRecordDescriptorFromWritableClasses(partialAggregateValueClass.getName()); + RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, + VLongWritable.class.getName()); + RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, + partialAggregateValueClassNames); IConfigurationFactory configurationFactory = new ConfigurationFactory(conf); IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory); IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( - vertexIdClass.getName(), vertexClass.getName()); - RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(), + conf, vertexIdClass.getName(), vertexClass.getName()); + RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), MsgList.class.getName()); - RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), messageValueClass.getName()); - RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(), + RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), vertexClass.getName()); - RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName()); + RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName()); TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec, recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, @@ -166,8 +167,8 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE /** * final aggregate write operator */ - IRecordDescriptorFactory aggRdFactory = DataflowUtils - .getWritableRecordDescriptorFactoryFromWritableClasses(partialAggregateValueClass.getName()); + IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( + conf, partialAggregateValueClassNames); FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec, configurationFactory, aggRdFactory, jobId); PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1); @@ -217,7 +218,8 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE /** * construct the materializing write operator */ - MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal); + MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal, + jobId, iteration); ClusterConfig.setLocationConstraint(spec, materialize); /** @@ -303,22 +305,22 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac Class> vertexIdClass = BspUtils.getVertexIndexClass(conf); Class vertexClass = BspUtils.getVertexClass(conf); Class messageValueClass = BspUtils.getMessageValueClass(conf); - Class partialAggregateValueClass = BspUtils.getPartialAggregateValueClass(conf); + String[] partialAggregateValueClassNames = BspUtils.getPartialAggregateValueClassNames(conf); JobSpecification spec = new JobSpecification(); /** * source aggregate */ int[] keyFields = new int[] { 0 }; - RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), messageValueClass.getName()); IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1]; comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);; - RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(), + RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), MsgList.class.getName()); - RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(), + RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), vertexClass.getName()); - RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName()); + RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName()); /** * construct empty tuple operator @@ -338,7 +340,7 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac * construct the materializing write operator */ MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal, - true); + true, jobId, iteration); ClusterConfig.setLocationConstraint(spec, materializeRead); /** @@ -359,13 +361,14 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac * construct index-join-function-update operator */ IFileSplitProvider fileSplitProvider = ClusterConfig.getFileSplitProvider(jobId, PRIMARY_INDEX); - RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(VLongWritable.class.getName()); - RecordDescriptor rdPartialAggregate = DataflowUtils - .getRecordDescriptorFromWritableClasses(partialAggregateValueClass.getName()); + RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, + VLongWritable.class.getName()); + RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, + partialAggregateValueClassNames); IConfigurationFactory configurationFactory = new ConfigurationFactory(conf); IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory); IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( - vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName()); + conf, vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName()); IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor( spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, @@ -421,7 +424,8 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac /** * construct the materializing write operator */ - MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal); + MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal, + jobId, iteration); ClusterConfig.setLocationConstraint(spec, materialize); /** construct runtime hook */ @@ -443,8 +447,8 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac /** * final aggregate write operator */ - IRecordDescriptorFactory aggRdFactory = DataflowUtils - .getWritableRecordDescriptorFactoryFromWritableClasses(partialAggregateValueClass.getName()); + IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( + conf, partialAggregateValueClassNames); FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec, configurationFactory, aggRdFactory, jobId); PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1); @@ -583,7 +587,7 @@ private JobSpecification generateSecondaryBTreeCheckpointLoad(int lastSuccessful } catch (Exception e) { throw new HyracksDataException(e); } - RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), MsgList.class.getName()); String[] readSchedule = ClusterConfig.getHdfsScheduler().getLocationConstraints(splits); HDFSReadOperatorDescriptor scanner = new HDFSReadOperatorDescriptor(spec, recordDescriptor, tmpJob, splits, @@ -660,7 +664,7 @@ private JobSpecification generateSecondaryBTreeCheckpoint(int lastSuccessfulIter /** * construct btree search operator */ - RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), msgListClass.getName()); IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1]; comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(0, vertexIdClass);; @@ -679,7 +683,7 @@ comparatorFactories, null, null, null, true, true, getIndexDataflowHelperFactory * construct write file operator */ IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( - vertexIdClass.getName(), MsgList.class.getName()); + conf, vertexIdClass.getName(), MsgList.class.getName()); HDFSFileWriteOperatorDescriptor writer = new HDFSFileWriteOperatorDescriptor(spec, job, inputRdFactory); ClusterConfig.setLocationConstraint(spec, writer); diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java index 2e83d2db6..d01c0690d 100644 --- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java +++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoin.java @@ -17,7 +17,6 @@ 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 edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper; import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory; @@ -79,7 +78,7 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE Class> vertexIdClass = BspUtils.getVertexIndexClass(conf); Class vertexClass = BspUtils.getVertexClass(conf); Class messageValueClass = BspUtils.getMessageValueClass(conf); - Class partialAggregateValueClass = BspUtils.getPartialAggregateValueClass(conf); + String[] partialAggregateValueClassNames = BspUtils.getPartialAggregateValueClassNames(conf); IConfigurationFactory confFactory = new ConfigurationFactory(conf); JobSpecification spec = new JobSpecification(); @@ -97,7 +96,7 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE /** * construct btree search function update operator */ - RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), vertexClass.getName()); IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1]; comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass);; @@ -107,18 +106,19 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE typeTraits[0] = new TypeTraits(false); typeTraits[1] = new TypeTraits(false); - RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(VLongWritable.class.getName()); - RecordDescriptor rdPartialAggregate = DataflowUtils - .getRecordDescriptorFromWritableClasses(partialAggregateValueClass.getName()); + RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, + VLongWritable.class.getName()); + RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, + partialAggregateValueClassNames); IConfigurationFactory configurationFactory = new ConfigurationFactory(conf); IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory); IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( - vertexIdClass.getName(), vertexClass.getName()); - RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + conf, vertexIdClass.getName(), vertexClass.getName()); + RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), messageValueClass.getName()); - RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(), + RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), vertexClass.getName()); - RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName()); + RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName()); TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec, recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, @@ -150,7 +150,7 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE /** * construct global group-by operator */ - RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(), + RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), MsgList.class.getName()); IClusteredAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory( conf, true, true); @@ -161,7 +161,8 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE /** * construct the materializing write operator */ - MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal); + MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal, + jobId, iteration); ClusterConfig.setLocationConstraint(spec, materialize); RuntimeHookOperatorDescriptor postSuperStep = new RuntimeHookOperatorDescriptor(spec, @@ -183,8 +184,8 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE /** * final aggregate write operator */ - IRecordDescriptorFactory aggRdFactory = DataflowUtils - .getWritableRecordDescriptorFactoryFromWritableClasses(partialAggregateValueClass.getName()); + IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( + conf, partialAggregateValueClassNames); FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec, configurationFactory, aggRdFactory, jobId); PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1); @@ -260,22 +261,22 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac Class> vertexIdClass = BspUtils.getVertexIndexClass(conf); Class vertexClass = BspUtils.getVertexClass(conf); Class messageValueClass = BspUtils.getMessageValueClass(conf); - Class partialAggregateValueClass = BspUtils.getPartialAggregateValueClass(conf); + String[] partialAggregateValueClassNames = BspUtils.getPartialAggregateValueClassNames(conf); JobSpecification spec = new JobSpecification(); /** * source aggregate */ int[] keyFields = new int[] { 0 }; - RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), messageValueClass.getName()); IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1]; comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass); - RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(), + RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), MsgList.class.getName()); - RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(), + RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), vertexClass.getName()); - RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName()); + RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName()); /** * construct empty tuple operator @@ -295,7 +296,7 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac * construct the materializing write operator */ MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal, - true); + true, jobId, iteration); ClusterConfig.setLocationConstraint(spec, materializeRead); /** @@ -309,13 +310,14 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac nullWriterFactories[0] = VertexIdNullWriterFactory.INSTANCE; nullWriterFactories[1] = MsgListNullWriterFactory.INSTANCE; - RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(VLongWritable.class.getName()); - RecordDescriptor rdPartialAggregate = DataflowUtils - .getRecordDescriptorFromWritableClasses(partialAggregateValueClass.getName()); + RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, + VLongWritable.class.getName()); + RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, + partialAggregateValueClassNames); IConfigurationFactory configurationFactory = new ConfigurationFactory(conf); IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory); IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( - vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName()); + conf, vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName()); IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor( spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, @@ -356,7 +358,8 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac /** * construct the materializing write operator */ - MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal); + MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal, + jobId, iteration); ClusterConfig.setLocationConstraint(spec, materialize); /** construct runtime hook */ @@ -378,8 +381,8 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac /** * final aggregate write operator */ - IRecordDescriptorFactory aggRdFactory = DataflowUtils - .getWritableRecordDescriptorFactoryFromWritableClasses(partialAggregateValueClass.getName()); + IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( + conf, partialAggregateValueClassNames); FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec, configurationFactory, aggRdFactory, jobId); PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1); diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java index 09c14a07e..4480b9713 100644 --- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java +++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSingleSort.java @@ -78,7 +78,7 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE Class> vertexIdClass = BspUtils.getVertexIndexClass(conf); Class vertexClass = BspUtils.getVertexClass(conf); Class messageValueClass = BspUtils.getMessageValueClass(conf); - Class partialAggregateValueClass = BspUtils.getPartialAggregateValueClass(conf); + String[] partialAggregateValueClassNames = BspUtils.getPartialAggregateValueClassNames(conf); IConfigurationFactory confFactory = new ConfigurationFactory(conf); JobSpecification spec = new JobSpecification(); @@ -96,7 +96,7 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE /** * construct btree search operator */ - RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), vertexClass.getName()); IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1]; comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass); @@ -109,18 +109,19 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE /** * construct compute operator */ - RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(VLongWritable.class.getName()); - RecordDescriptor rdPartialAggregate = DataflowUtils - .getRecordDescriptorFromWritableClasses(partialAggregateValueClass.getName()); + RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, + VLongWritable.class.getName()); + RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, + partialAggregateValueClassNames); IConfigurationFactory configurationFactory = new ConfigurationFactory(conf); IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory); IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( - vertexIdClass.getName(), vertexClass.getName()); - RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + conf, vertexIdClass.getName(), vertexClass.getName()); + RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), messageValueClass.getName()); - RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(), + RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), vertexClass.getName()); - RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName()); + RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName()); TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec, recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, @@ -144,7 +145,7 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE /** * construct global group-by operator */ - RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(), + RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), MsgList.class.getName()); IClusteredAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory( conf, true, false); @@ -155,7 +156,8 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE /** * construct the materializing write operator */ - MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal); + MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal, + jobId, iteration); ClusterConfig.setLocationConstraint(spec, materialize); RuntimeHookOperatorDescriptor postSuperStep = new RuntimeHookOperatorDescriptor(spec, @@ -176,8 +178,8 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE /** * final aggregate write operator */ - IRecordDescriptorFactory aggRdFactory = DataflowUtils - .getWritableRecordDescriptorFactoryFromWritableClasses(partialAggregateValueClass.getName()); + IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( + conf, partialAggregateValueClassNames); FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec, configurationFactory, aggRdFactory, jobId); PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1); @@ -250,22 +252,22 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac Class> vertexIdClass = BspUtils.getVertexIndexClass(conf); Class vertexClass = BspUtils.getVertexClass(conf); Class messageValueClass = BspUtils.getMessageValueClass(conf); - Class partialAggregateValueClass = BspUtils.getPartialAggregateValueClass(conf); + String[] partialAggregateValueClassNames = BspUtils.getPartialAggregateValueClassNames(conf); JobSpecification spec = new JobSpecification(); /** * source aggregate */ int[] keyFields = new int[] { 0 }; - RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), messageValueClass.getName()); IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1]; comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass); - RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(), + RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), MsgList.class.getName()); - RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(), + RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), vertexClass.getName()); - RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName()); + RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName()); /** * construct empty tuple operator @@ -285,7 +287,7 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac * construct the materializing write operator */ MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal, - true); + true, jobId, iteration); ClusterConfig.setLocationConstraint(spec, materializeRead); /** @@ -299,13 +301,14 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac nullWriterFactories[0] = VertexIdNullWriterFactory.INSTANCE; nullWriterFactories[1] = MsgListNullWriterFactory.INSTANCE; - RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(VLongWritable.class.getName()); - RecordDescriptor rdPartialAggregate = DataflowUtils - .getRecordDescriptorFromWritableClasses(partialAggregateValueClass.getName()); + RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, + VLongWritable.class.getName()); + RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, + partialAggregateValueClassNames); IConfigurationFactory configurationFactory = new ConfigurationFactory(conf); IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory); IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( - vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName()); + conf, vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName()); IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor( spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, @@ -338,7 +341,8 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac /** * construct the materializing write operator */ - MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal); + MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal, + jobId, iteration); ClusterConfig.setLocationConstraint(spec, materialize); /** construct runtime hook */ @@ -360,8 +364,8 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac /** * final aggregate write operator */ - IRecordDescriptorFactory aggRdFactory = DataflowUtils - .getWritableRecordDescriptorFactoryFromWritableClasses(partialAggregateValueClass.getName()); + IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( + conf, partialAggregateValueClassNames); FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec, configurationFactory, aggRdFactory, jobId); PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1); diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java index 379147c20..7ca771c64 100644 --- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java +++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/jobgen/JobGenOuterJoinSort.java @@ -78,7 +78,7 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE Class> vertexIdClass = BspUtils.getVertexIndexClass(conf); Class vertexClass = BspUtils.getVertexClass(conf); Class messageValueClass = BspUtils.getMessageValueClass(conf); - Class partialAggregateValueClass = BspUtils.getPartialAggregateValueClass(conf); + String[] partialAggregateValueClassNames = BspUtils.getPartialAggregateValueClassNames(conf); IConfigurationFactory confFactory = new ConfigurationFactory(conf); JobSpecification spec = new JobSpecification(); @@ -96,7 +96,7 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE /** * construct btree search function update operator */ - RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + RecordDescriptor recordDescriptor = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), vertexClass.getName()); IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1]; comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass); @@ -106,18 +106,19 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE typeTraits[0] = new TypeTraits(false); typeTraits[1] = new TypeTraits(false); - RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(VLongWritable.class.getName()); - RecordDescriptor rdPartialAggregate = DataflowUtils - .getRecordDescriptorFromWritableClasses(partialAggregateValueClass.getName()); + RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, + VLongWritable.class.getName()); + RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, + partialAggregateValueClassNames); IConfigurationFactory configurationFactory = new ConfigurationFactory(conf); IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory); IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( - vertexIdClass.getName(), vertexClass.getName()); - RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + conf, vertexIdClass.getName(), vertexClass.getName()); + RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), messageValueClass.getName()); - RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(), + RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), vertexClass.getName()); - RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName()); + RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName()); TreeSearchFunctionUpdateOperatorDescriptor scanner = new TreeSearchFunctionUpdateOperatorDescriptor(spec, recordDescriptor, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, @@ -157,7 +158,7 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE /** * construct global group-by operator */ - RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(), + RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), MsgList.class.getName()); IClusteredAggregatorDescriptorFactory aggregatorFactoryFinal = DataflowUtils.getAccumulatingAggregatorFactory( conf, true, true); @@ -168,7 +169,8 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE /** * construct the materializing write operator */ - MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal); + MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal, + jobId, iteration); ClusterConfig.setLocationConstraint(spec, materialize); RuntimeHookOperatorDescriptor postSuperStep = new RuntimeHookOperatorDescriptor(spec, @@ -190,8 +192,8 @@ protected JobSpecification generateFirstIteration(int iteration) throws HyracksE /** * final aggregate write operator */ - IRecordDescriptorFactory aggRdFactory = DataflowUtils - .getWritableRecordDescriptorFactoryFromWritableClasses(partialAggregateValueClass.getName()); + IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( + conf, partialAggregateValueClassNames); FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec, configurationFactory, aggRdFactory, jobId); PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1); @@ -264,22 +266,22 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac Class> vertexIdClass = BspUtils.getVertexIndexClass(conf); Class vertexClass = BspUtils.getVertexClass(conf); Class messageValueClass = BspUtils.getMessageValueClass(conf); - Class partialAggregateValueClass = BspUtils.getPartialAggregateValueClass(conf); + String[] partialAggregateValueClassNames = BspUtils.getPartialAggregateValueClassNames(conf); JobSpecification spec = new JobSpecification(); /** * source aggregate */ int[] keyFields = new int[] { 0 }; - RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses( + RecordDescriptor rdUnnestedMessage = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), messageValueClass.getName()); IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[1]; comparatorFactories[0] = JobGenUtil.getIBinaryComparatorFactory(iteration, vertexIdClass); - RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(), + RecordDescriptor rdFinal = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), MsgList.class.getName()); - RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(vertexIdClass.getName(), + RecordDescriptor rdInsert = DataflowUtils.getRecordDescriptorFromKeyValueClasses(conf, vertexIdClass.getName(), vertexClass.getName()); - RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(vertexIdClass.getName()); + RecordDescriptor rdDelete = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, vertexIdClass.getName()); /** * construct empty tuple operator @@ -299,7 +301,7 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac * construct the materializing write operator */ MaterializingReadOperatorDescriptor materializeRead = new MaterializingReadOperatorDescriptor(spec, rdFinal, - true); + true, jobId, iteration); ClusterConfig.setLocationConstraint(spec, materializeRead); /** @@ -313,13 +315,14 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac nullWriterFactories[0] = VertexIdNullWriterFactory.INSTANCE; nullWriterFactories[1] = MsgListNullWriterFactory.INSTANCE; - RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(VLongWritable.class.getName()); - RecordDescriptor rdPartialAggregate = DataflowUtils - .getRecordDescriptorFromWritableClasses(partialAggregateValueClass.getName()); + RecordDescriptor rdDummy = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, + VLongWritable.class.getName()); + RecordDescriptor rdPartialAggregate = DataflowUtils.getRecordDescriptorFromWritableClasses(conf, + partialAggregateValueClassNames); IConfigurationFactory configurationFactory = new ConfigurationFactory(conf); IRuntimeHookFactory preHookFactory = new RuntimeHookFactory(configurationFactory); IRecordDescriptorFactory inputRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( - vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName()); + conf, vertexIdClass.getName(), MsgList.class.getName(), vertexIdClass.getName(), vertexClass.getName()); IndexNestedLoopJoinFunctionUpdateOperatorDescriptor join = new IndexNestedLoopJoinFunctionUpdateOperatorDescriptor( spec, storageManagerInterface, lcManagerProvider, fileSplitProvider, typeTraits, comparatorFactories, @@ -368,7 +371,8 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac /** * construct the materializing write operator */ - MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal); + MaterializingWriteOperatorDescriptor materialize = new MaterializingWriteOperatorDescriptor(spec, rdFinal, + jobId, iteration); ClusterConfig.setLocationConstraint(spec, materialize); /** construct runtime hook */ @@ -390,8 +394,8 @@ protected JobSpecification generateNonFirstIteration(int iteration) throws Hyrac /** * final aggregate write operator */ - IRecordDescriptorFactory aggRdFactory = DataflowUtils - .getWritableRecordDescriptorFactoryFromWritableClasses(partialAggregateValueClass.getName()); + IRecordDescriptorFactory aggRdFactory = DataflowUtils.getWritableRecordDescriptorFactoryFromWritableClasses( + conf, partialAggregateValueClassNames); FinalAggregateOperatorDescriptor finalAggregator = new FinalAggregateOperatorDescriptor(spec, configurationFactory, aggRdFactory, jobId); PartitionConstraintHelper.addPartitionCountConstraint(spec, finalAggregator, 1); diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/WritableRecordDescriptorFactory.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/WritableRecordDescriptorFactory.java index a67c25983..68e3ba7aa 100644 --- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/WritableRecordDescriptorFactory.java +++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/runtime/touchpoint/WritableRecordDescriptorFactory.java @@ -14,25 +14,32 @@ */ package edu.uci.ics.pregelix.core.runtime.touchpoint; +import org.apache.hadoop.conf.Configuration; + import edu.uci.ics.hyracks.api.context.IHyracksTaskContext; 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.pregelix.core.hadoop.config.ConfigurationFactory; import edu.uci.ics.pregelix.core.util.DataflowUtils; +import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory; import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory; public class WritableRecordDescriptorFactory implements IRecordDescriptorFactory { private static final long serialVersionUID = 1L; private String[] fieldClasses; + private IConfigurationFactory confFactory; - public WritableRecordDescriptorFactory(String... fieldClasses) { + public WritableRecordDescriptorFactory(Configuration conf, String... fieldClasses) { this.fieldClasses = fieldClasses; + this.confFactory = new ConfigurationFactory(conf); } @Override public RecordDescriptor createRecordDescriptor(IHyracksTaskContext ctx) throws HyracksDataException { try { - return DataflowUtils.getRecordDescriptorFromWritableClasses(ctx, fieldClasses); + Configuration conf = confFactory.createConfiguration(ctx); + return DataflowUtils.getRecordDescriptorFromWritableClasses(ctx, conf, fieldClasses); } catch (HyracksException e) { throw new HyracksDataException(e); } diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/DataflowUtils.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/DataflowUtils.java index 3e0110963..3a2241ba8 100644 --- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/DataflowUtils.java +++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/DataflowUtils.java @@ -38,14 +38,14 @@ public enum AggregationMode { } @SuppressWarnings("unchecked") - public static RecordDescriptor getRecordDescriptorFromKeyValueClasses(String className1, String className2) - throws HyracksException { + public static RecordDescriptor getRecordDescriptorFromKeyValueClasses(Configuration conf, String className1, + String className2) throws HyracksException { RecordDescriptor recordDescriptor = null; try { ClassLoader loader = DataflowUtils.class.getClassLoader(); recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor( (Class) loader.loadClass(className1), - (Class) loader.loadClass(className2)); + (Class) loader.loadClass(className2), conf); } catch (ClassNotFoundException cnfe) { throw new HyracksException(cnfe); } @@ -53,15 +53,16 @@ public static RecordDescriptor getRecordDescriptorFromKeyValueClasses(String cla } @SuppressWarnings({ "unchecked", "rawtypes" }) - public static RecordDescriptor getRecordDescriptorFromWritableClasses(String... classNames) throws HyracksException { + public static RecordDescriptor getRecordDescriptorFromWritableClasses(Configuration conf, String... classNames) + throws HyracksException { RecordDescriptor recordDescriptor = null; ISerializerDeserializer[] serdes = new ISerializerDeserializer[classNames.length]; ClassLoader loader = DataflowUtils.class.getClassLoader(); try { int i = 0; for (String className : classNames) - serdes[i++] = DatatypeHelper.createSerializerDeserializer((Class) loader - .loadClass(className)); + serdes[i++] = DatatypeHelper.createSerializerDeserializer( + (Class) loader.loadClass(className), conf); } catch (ClassNotFoundException cnfe) { throw new HyracksException(cnfe); } @@ -69,9 +70,9 @@ public static RecordDescriptor getRecordDescriptorFromWritableClasses(String... return recordDescriptor; } - public static IRecordDescriptorFactory getWritableRecordDescriptorFactoryFromWritableClasses(String... classNames) - throws HyracksException { - IRecordDescriptorFactory rdFactory = new WritableRecordDescriptorFactory(classNames); + public static IRecordDescriptorFactory getWritableRecordDescriptorFactoryFromWritableClasses(Configuration conf, + String... classNames) throws HyracksException { + IRecordDescriptorFactory rdFactory = new WritableRecordDescriptorFactory(conf, classNames); return rdFactory; } @@ -85,13 +86,13 @@ public static IClusteredAggregatorDescriptorFactory getAccumulatingAggregatorFac } @SuppressWarnings("unchecked") - public static RecordDescriptor getRecordDescriptorFromKeyValueClasses(IHyracksTaskContext ctx, String className1, - String className2) throws HyracksException { + public static RecordDescriptor getRecordDescriptorFromKeyValueClasses(IHyracksTaskContext ctx, Configuration conf, + String className1, String className2) throws HyracksException { RecordDescriptor recordDescriptor = null; try { recordDescriptor = DatatypeHelper.createKeyValueRecordDescriptor((Class) ctx .getJobletContext().loadClass(className1), (Class) ctx.getJobletContext() - .loadClass(className2)); + .loadClass(className2), conf); } catch (Exception cnfe) { throw new HyracksException(cnfe); } @@ -99,15 +100,17 @@ public static RecordDescriptor getRecordDescriptorFromKeyValueClasses(IHyracksTa } @SuppressWarnings({ "unchecked", "rawtypes" }) - public static RecordDescriptor getRecordDescriptorFromWritableClasses(IHyracksTaskContext ctx, String... classNames) - throws HyracksException { + public static RecordDescriptor getRecordDescriptorFromWritableClasses(IHyracksTaskContext ctx, Configuration conf, + String... classNames) throws HyracksException { RecordDescriptor recordDescriptor = null; ISerializerDeserializer[] serdes = new ISerializerDeserializer[classNames.length]; try { int i = 0; - for (String className : classNames) - serdes[i++] = DatatypeHelper.createSerializerDeserializer((Class) ctx - .getJobletContext().loadClass(className)); + for (String className : classNames) { + Class c = (Class) ctx.getJobletContext().loadClass(className); + serdes[i++] = DatatypeHelper.createSerializerDeserializer(c, conf); + //System.out.println("thread " + Thread.currentThread().getId() + " after creating serde " + c.getClassLoader()); + } } catch (Exception cnfe) { throw new HyracksException(cnfe); } diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java index aabd4ba7f..70de9ed55 100644 --- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java +++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java @@ -62,7 +62,7 @@ public static void init() throws Exception { ccConfig.jobHistorySize = 1; ccConfig.profileDumpPeriod = -1; ccConfig.heartbeatPeriod = 50; - ccConfig.maxHeartbeatLapsePeriods = 15; + ccConfig.maxHeartbeatLapsePeriods = 10; // cluster controller cc = new ClusterControllerService(ccConfig); diff --git a/pregelix/pregelix-dataflow-std-base/pom.xml b/pregelix/pregelix-dataflow-std-base/pom.xml index d4c0ee618..77d75bfb4 100644 --- a/pregelix/pregelix-dataflow-std-base/pom.xml +++ b/pregelix/pregelix-dataflow-std-base/pom.xml @@ -1,28 +1,24 @@ - - + + 4.0.0 pregelix-dataflow-std-base jar pregelix-dataflow-std-base - edu.uci.ics.hyracks - pregelix - 0.2.10-SNAPSHOT - + edu.uci.ics.hyracks + pregelix + 0.2.10-SNAPSHOT + @@ -58,7 +54,7 @@ org.apache.maven.plugins maven-clean-plugin - 2.4.1 + 2.4.1 @@ -92,6 +88,20 @@ jar compile + + edu.uci.ics.hyracks + hyracks-hdfs-core + 0.2.10-SNAPSHOT + jar + compile + + + edu.uci.ics.hyracks + hyracks-storage-am-common + 0.2.10-SNAPSHOT + jar + compile + edu.uci.ics.hyracks hyracks-api diff --git a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/ISerializerDeserializerFactory.java b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/ISerializerDeserializerFactory.java index 1fdd0b606..894cba528 100644 --- a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/ISerializerDeserializerFactory.java +++ b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/ISerializerDeserializerFactory.java @@ -16,10 +16,12 @@ import java.io.Serializable; +import org.apache.hadoop.conf.Configuration; + import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer; public interface ISerializerDeserializerFactory extends Serializable { - public ISerializerDeserializer getSerializerDeserializer(); + public ISerializerDeserializer getSerializerDeserializer(Configuration conf); } diff --git a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IUpdateFunction.java b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IUpdateFunction.java index b8ba7bd5d..081b3bcd1 100644 --- a/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IUpdateFunction.java +++ b/pregelix/pregelix-dataflow-std-base/src/main/java/edu/uci/ics/pregelix/dataflow/std/base/IUpdateFunction.java @@ -18,17 +18,18 @@ import edu.uci.ics.hyracks.api.exceptions.HyracksDataException; import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder; import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference; +import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor; public interface IUpdateFunction extends IFunction { - /** - * update the tuple pointed by tupleRef called after process, - * one-input-tuple-at-a-time - * - * @param tupleRef - * @throws HyracksDataException - */ - public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb) - throws HyracksDataException; + /** + * update the tuple pointed by tupleRef called after process, + * one-input-tuple-at-a-time + * + * @param tupleRef + * @throws HyracksDataException + */ + public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb, IIndexCursor cursor) + throws HyracksDataException; } diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java index 2332188fe..b5a292702 100644 --- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java +++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinFunctionUpdateOperatorNodePushable.java @@ -34,6 +34,7 @@ import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex; import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor; import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper; +import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper; import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback; import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator; import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference; @@ -43,6 +44,7 @@ import edu.uci.ics.pregelix.dataflow.util.CopyUpdateUtil; import edu.uci.ics.pregelix.dataflow.util.FunctionProxy; import edu.uci.ics.pregelix.dataflow.util.SearchKeyTupleReference; +import edu.uci.ics.pregelix.dataflow.util.StorageType; import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer; public class IndexNestedLoopJoinFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable { @@ -68,6 +70,7 @@ public class IndexNestedLoopJoinFunctionUpdateOperatorNodePushable extends Abstr private ArrayTupleBuilder cloneUpdateTb; private final UpdateBuffer updateBuffer; private final SearchKeyTupleReference tempTupleReference = new SearchKeyTupleReference(); + private final StorageType storageType; public IndexNestedLoopJoinFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward, @@ -77,6 +80,11 @@ public IndexNestedLoopJoinFunctionUpdateOperatorNodePushable(AbstractTreeIndexOp throws HyracksDataException { treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper( opDesc, ctx, partition); + if (treeIndexOpHelper instanceof TreeIndexDataflowHelper) { + storageType = StorageType.TreeIndex; + } else { + storageType = StorageType.LSMIndex; + } this.lowKeyInclusive = lowKeyInclusive; this.highKeyInclusive = highKeyInclusive; this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0); @@ -96,7 +104,7 @@ public IndexNestedLoopJoinFunctionUpdateOperatorNodePushable(AbstractTreeIndexOp } protected void setCursor() { - cursor = indexAccessor.createSearchCursor(); + cursor = indexAccessor.createSearchCursor(true); } @Override @@ -146,7 +154,7 @@ public void open() throws HyracksDataException { cloneUpdateTb = new ArrayTupleBuilder(index.getFieldCount()); updateBuffer.setFieldCount(index.getFieldCount()); } catch (Exception e) { - treeIndexOpHelper.close(); + closeResource(); throw new HyracksDataException(e); } } @@ -170,13 +178,13 @@ private void writeSearchResults(IFrameTupleAccessor leftAccessor, int tIndex) th /** * call the update function */ - functionProxy.functionCall(leftAccessor, tIndex, indexEntryTuple, cloneUpdateTb); + functionProxy.functionCall(leftAccessor, tIndex, indexEntryTuple, cloneUpdateTb, cursor); /** * doing copy update */ CopyUpdateUtil.copyUpdate(tempTupleReference, indexEntryTuple, updateBuffer, cloneUpdateTb, indexAccessor, - cursor, rangePred, false); + cursor, rangePred, false, storageType); } } @@ -204,6 +212,7 @@ public void nextFrame(ByteBuffer buffer) throws HyracksDataException { writeSearchResults(accessor, i); } } catch (Exception e) { + closeResource(); throw new HyracksDataException(e); } } @@ -230,6 +239,11 @@ public void close() throws HyracksDataException { @Override public void fail() throws HyracksDataException { + closeResource(); + populateFailure(); + } + + private void closeResource() throws HyracksDataException { try { cursor.close(); } catch (Exception e) { @@ -237,6 +251,9 @@ public void fail() throws HyracksDataException { } finally { treeIndexOpHelper.close(); } + } + + private void populateFailure() throws HyracksDataException { for (IFrameWriter writer : writers) { writer.fail(); } diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorNodePushable.java index 2ddca903f..b5b21b620 100644 --- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorNodePushable.java +++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopJoinOperatorNodePushable.java @@ -82,7 +82,7 @@ public IndexNestedLoopJoinOperatorNodePushable(AbstractTreeIndexOperatorDescript } protected void setCursor() { - cursor = indexAccessor.createSearchCursor(); + cursor = indexAccessor.createSearchCursor(false); } @Override diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java index 3a36ab442..2a7fedec0 100644 --- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java +++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable.java @@ -36,6 +36,7 @@ import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex; import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor; import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper; +import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper; import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback; import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator; import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference; @@ -45,6 +46,7 @@ import edu.uci.ics.pregelix.dataflow.util.CopyUpdateUtil; import edu.uci.ics.pregelix.dataflow.util.FunctionProxy; import edu.uci.ics.pregelix.dataflow.util.SearchKeyTupleReference; +import edu.uci.ics.pregelix.dataflow.util.StorageType; import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer; public class IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable extends @@ -79,6 +81,7 @@ public class IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable ext private ArrayTupleBuilder cloneUpdateTb; private final UpdateBuffer updateBuffer; private final SearchKeyTupleReference tempTupleReference = new SearchKeyTupleReference(); + private final StorageType storageType; public IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward, @@ -88,6 +91,11 @@ public IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable(AbstractT inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0); treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper( opDesc, ctx, partition); + if (treeIndexOpHelper instanceof TreeIndexDataflowHelper) { + storageType = StorageType.TreeIndex; + } else { + storageType = StorageType.LSMIndex; + } this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0); if (lowKeyFields != null && lowKeyFields.length > 0) { @@ -107,7 +115,7 @@ public IndexNestedLoopRightOuterJoinFunctionUpdateOperatorNodePushable(AbstractT } protected void setCursor() { - cursor = indexAccessor.createSearchCursor(); + cursor = indexAccessor.createSearchCursor(true); } @Override @@ -177,7 +185,7 @@ public void open() throws HyracksDataException { cloneUpdateTb = new ArrayTupleBuilder(index.getFieldCount()); updateBuffer.setFieldCount(index.getFieldCount()); } catch (Exception e) { - treeIndexOpHelper.close(); + closeResource(); throw new HyracksDataException(e); } } @@ -205,6 +213,7 @@ public void nextFrame(ByteBuffer buffer) throws HyracksDataException { } } } catch (Exception e) { + closeResource(); throw new HyracksDataException(e); } } @@ -254,6 +263,11 @@ public void close() throws HyracksDataException { @Override public void fail() throws HyracksDataException { + closeResource(); + populateFailure(); + } + + private void closeResource() throws HyracksDataException { try { cursor.close(); } catch (Exception e) { @@ -261,6 +275,9 @@ public void fail() throws HyracksDataException { } finally { treeIndexOpHelper.close(); } + } + + private void populateFailure() throws HyracksDataException { for (IFrameWriter writer : writers) { writer.fail(); } @@ -287,13 +304,13 @@ private void writeResults(IFrameTupleAccessor leftAccessor, int tIndex, ITupleRe /** * function call */ - functionProxy.functionCall(leftAccessor, tIndex, indexEntryTuple, cloneUpdateTb); + functionProxy.functionCall(leftAccessor, tIndex, indexEntryTuple, cloneUpdateTb, cursor); /** * doing clone update */ CopyUpdateUtil.copyUpdate(tempTupleReference, indexEntryTuple, updateBuffer, cloneUpdateTb, indexAccessor, - cursor, rangePred, true); + cursor, rangePred, true, storageType); } /** write result for outer case */ @@ -301,11 +318,11 @@ private void writeResults(ITupleReference frameTuple) throws Exception { /** * function call */ - functionProxy.functionCall(nullTupleBuilder, frameTuple, cloneUpdateTb); + functionProxy.functionCall(nullTupleBuilder, frameTuple, cloneUpdateTb, cursor); //doing clone update CopyUpdateUtil.copyUpdate(tempTupleReference, frameTuple, updateBuffer, cloneUpdateTb, indexAccessor, cursor, - rangePred, true); + rangePred, true, storageType); } @Override diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java index e64e9cc06..65398282a 100644 --- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java +++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopRightOuterJoinOperatorNodePushable.java @@ -89,7 +89,7 @@ public IndexNestedLoopRightOuterJoinOperatorNodePushable(AbstractTreeIndexOperat } protected void setCursor() { - cursor = indexAccessor.createSearchCursor(); + cursor = indexAccessor.createSearchCursor(false); } @Override diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java index 8d9728930..fe27029fd 100644 --- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java +++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable.java @@ -34,6 +34,7 @@ import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex; import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor; import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper; +import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper; import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback; import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator; import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference; @@ -43,6 +44,7 @@ import edu.uci.ics.pregelix.dataflow.util.CopyUpdateUtil; import edu.uci.ics.pregelix.dataflow.util.FunctionProxy; import edu.uci.ics.pregelix.dataflow.util.SearchKeyTupleReference; +import edu.uci.ics.pregelix.dataflow.util.StorageType; import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer; public class IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable { @@ -71,6 +73,7 @@ public class IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable extends A private ArrayTupleBuilder cloneUpdateTb; private UpdateBuffer updateBuffer; private final SearchKeyTupleReference tempTupleReference = new SearchKeyTupleReference(); + private final StorageType storageType; public IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward, @@ -79,6 +82,11 @@ public IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable(AbstractTreeInd IRecordDescriptorFactory inputRdFactory, int outputArity) throws HyracksDataException { treeIndexOpHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper( opDesc, ctx, partition); + if (treeIndexOpHelper instanceof TreeIndexDataflowHelper) { + storageType = StorageType.TreeIndex; + } else { + storageType = StorageType.LSMIndex; + } this.isForward = isForward; this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0); @@ -98,7 +106,7 @@ public IndexNestedLoopSetUnionFunctionUpdateOperatorNodePushable(AbstractTreeInd } protected void setCursor() { - cursor = indexAccessor.createSearchCursor(); + cursor = indexAccessor.createSearchCursor(true); } @Override @@ -140,7 +148,7 @@ public void open() throws HyracksDataException { cloneUpdateTb = new ArrayTupleBuilder(index.getFieldCount()); updateBuffer.setFieldCount(index.getFieldCount()); } catch (Exception e) { - treeIndexOpHelper.close(); + closeResource(); throw new HyracksDataException(e); } } @@ -173,6 +181,7 @@ public void nextFrame(ByteBuffer buffer) throws HyracksDataException { } } } catch (Exception e) { + closeResource(); throw new HyracksDataException(e); } } @@ -219,6 +228,17 @@ public void close() throws HyracksDataException { @Override public void fail() throws HyracksDataException { + closeResource(); + populateFailure(); + } + + private void populateFailure() throws HyracksDataException { + for (IFrameWriter writer : writers) { + writer.fail(); + } + } + + private void closeResource() throws HyracksDataException { try { cursor.close(); } catch (Exception e) { @@ -226,9 +246,6 @@ public void fail() throws HyracksDataException { } finally { treeIndexOpHelper.close(); } - for (IFrameWriter writer : writers) { - writer.fail(); - } } /** compare tuples */ @@ -238,21 +255,21 @@ private int compare(ITupleReference left, ITupleReference right) throws Exceptio /** write the right result */ private void writeRightResults(ITupleReference frameTuple) throws Exception { - functionProxy.functionCall(frameTuple, cloneUpdateTb); + functionProxy.functionCall(frameTuple, cloneUpdateTb, cursor); //doing clone update CopyUpdateUtil.copyUpdate(tempTupleReference, frameTuple, updateBuffer, cloneUpdateTb, indexAccessor, cursor, - rangePred, true); + rangePred, true, storageType); } /** write the left result */ private void writeLeftResults(IFrameTupleAccessor leftAccessor, int tIndex, ITupleReference frameTuple) throws Exception { - functionProxy.functionCall(leftAccessor, tIndex, frameTuple, cloneUpdateTb); + functionProxy.functionCall(leftAccessor, tIndex, frameTuple, cloneUpdateTb, cursor); //doing clone update CopyUpdateUtil.copyUpdate(tempTupleReference, frameTuple, updateBuffer, cloneUpdateTb, indexAccessor, cursor, - rangePred, true); + rangePred, true, storageType); } @Override diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java index 86a211fe3..1c9fce60e 100644 --- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java +++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/IndexNestedLoopSetUnionOperatorNodePushable.java @@ -82,7 +82,7 @@ public IndexNestedLoopSetUnionOperatorNodePushable(AbstractTreeIndexOperatorDesc } protected void setCursor() { - cursor = indexAccessor.createSearchCursor(); + cursor = indexAccessor.createSearchCursor(false); } @Override diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java index dbdbfa266..f955831b9 100644 --- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java +++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/std/TreeSearchFunctionUpdateOperatorNodePushable.java @@ -35,6 +35,7 @@ import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame; import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor; import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDataflowHelper; +import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper; import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback; import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator; import edu.uci.ics.hyracks.storage.am.common.tuples.PermutingFrameTupleReference; @@ -44,6 +45,7 @@ import edu.uci.ics.pregelix.dataflow.util.CopyUpdateUtil; import edu.uci.ics.pregelix.dataflow.util.FunctionProxy; import edu.uci.ics.pregelix.dataflow.util.SearchKeyTupleReference; +import edu.uci.ics.pregelix.dataflow.util.StorageType; import edu.uci.ics.pregelix.dataflow.util.UpdateBuffer; public class TreeSearchFunctionUpdateOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable { @@ -77,6 +79,7 @@ public class TreeSearchFunctionUpdateOperatorNodePushable extends AbstractUnaryI private ArrayTupleBuilder cloneUpdateTb; private final UpdateBuffer updateBuffer; private final SearchKeyTupleReference tempTupleReference = new SearchKeyTupleReference(); + private final StorageType storageType; public TreeSearchFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition, IRecordDescriptorProvider recordDescProvider, boolean isForward, @@ -86,6 +89,11 @@ public TreeSearchFunctionUpdateOperatorNodePushable(AbstractTreeIndexOperatorDes throws HyracksDataException { treeIndexHelper = (IndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper( opDesc, ctx, partition); + if (treeIndexHelper instanceof TreeIndexDataflowHelper) { + storageType = StorageType.TreeIndex; + } else { + storageType = StorageType.LSMIndex; + } this.isForward = isForward; this.lowKeyInclusive = lowKeyInclusive; this.highKeyInclusive = highKeyInclusive; @@ -158,24 +166,24 @@ public void open() throws HyracksDataException { cloneUpdateTb = new ArrayTupleBuilder(index.getFieldCount()); updateBuffer.setFieldCount(index.getFieldCount()); } catch (Exception e) { - treeIndexHelper.close(); + closeResource(); throw new HyracksDataException(e); } } protected void setCursor() { - cursor = indexAccessor.createSearchCursor(); + cursor = indexAccessor.createSearchCursor(true); } protected void writeSearchResults() throws Exception { while (cursor.hasNext()) { cursor.next(); ITupleReference tuple = cursor.getTuple(); - functionProxy.functionCall(tuple, cloneUpdateTb); + functionProxy.functionCall(tuple, cloneUpdateTb, cursor); //doing clone update CopyUpdateUtil.copyUpdate(tempTupleReference, tuple, updateBuffer, cloneUpdateTb, indexAccessor, cursor, - rangePred, true); + rangePred, true, storageType); } } @@ -198,6 +206,7 @@ public void nextFrame(ByteBuffer buffer) throws HyracksDataException { writeSearchResults(); } } catch (Exception e) { + closeResource(); throw new HyracksDataException(e); } } @@ -224,6 +233,17 @@ public void close() throws HyracksDataException { @Override public void fail() throws HyracksDataException { + closeResource(); + populateFailure(); + } + + private void populateFailure() throws HyracksDataException { + for (IFrameWriter writer : writers) { + writer.fail(); + } + } + + private void closeResource() throws HyracksDataException { try { cursor.close(); } catch (Exception e) { @@ -231,9 +251,6 @@ public void fail() throws HyracksDataException { } finally { treeIndexHelper.close(); } - for (IFrameWriter writer : writers) { - writer.fail(); - } } @Override diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/CopyUpdateUtil.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/CopyUpdateUtil.java index ed3dfb593..0ff3f043f 100644 --- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/CopyUpdateUtil.java +++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/CopyUpdateUtil.java @@ -27,7 +27,8 @@ public class CopyUpdateUtil { public static void copyUpdate(SearchKeyTupleReference tempTupleReference, ITupleReference frameTuple, UpdateBuffer updateBuffer, ArrayTupleBuilder cloneUpdateTb, IIndexAccessor indexAccessor, - IIndexCursor cursor, RangePredicate rangePred, boolean scan) throws HyracksDataException, IndexException { + IIndexCursor cursor, RangePredicate rangePred, boolean scan, StorageType type) throws HyracksDataException, + IndexException { if (cloneUpdateTb.getSize() > 0) { if (!updateBuffer.appendTuple(cloneUpdateTb)) { tempTupleReference.reset(frameTuple.getFieldData(0), frameTuple.getFieldStart(0), @@ -40,7 +41,7 @@ public static void copyUpdate(SearchKeyTupleReference tempTupleReference, ITuple if (!updateBuffer.appendTuple(cloneUpdateTb)) { throw new HyracksDataException("cannot append tuple builder!"); } - //search again and recover the cursor + //search again and recover the cursor to the exact point as the one before it is closed cursor.reset(); rangePred.setLowKey(tempTupleReference, true); if (scan) { diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java index a1e5b8643..5579a7712 100644 --- a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java +++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/FunctionProxy.java @@ -22,6 +22,7 @@ import edu.uci.ics.hyracks.api.exceptions.HyracksDataException; import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder; import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference; +import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor; import edu.uci.ics.pregelix.dataflow.std.base.IRecordDescriptorFactory; import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory; import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunction; @@ -56,10 +57,10 @@ public FunctionProxy(IHyracksTaskContext ctx, IUpdateFunctionFactory functionFac * @throws HyracksDataException */ public void functionOpen() throws HyracksDataException { + ctxCL = Thread.currentThread().getContextClassLoader(); + Thread.currentThread().setContextClassLoader(ctx.getJobletContext().getClassLoader()); inputRd = inputRdFactory.createRecordDescriptor(ctx); tupleDe = new TupleDeserializer(inputRd); - ctxCL = Thread.currentThread().getContextClassLoader(); - Thread.currentThread().setContextClassLoader(this.getClass().getClassLoader()); for (IFrameWriter writer : writers) { writer.open(); } @@ -80,10 +81,10 @@ public void functionOpen() throws HyracksDataException { * @throws HyracksDataException */ public void functionCall(IFrameTupleAccessor leftAccessor, int leftTupleIndex, ITupleReference right, - ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException { + ArrayTupleBuilder cloneUpdateTb, IIndexCursor cursor) throws HyracksDataException { Object[] tuple = tupleDe.deserializeRecord(leftAccessor, leftTupleIndex, right); function.process(tuple); - function.update(right, cloneUpdateTb); + function.update(right, cloneUpdateTb, cursor); } /** @@ -92,10 +93,11 @@ public void functionCall(IFrameTupleAccessor leftAccessor, int leftTupleIndex, I * @param updateRef * @throws HyracksDataException */ - public void functionCall(ITupleReference updateRef, ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException { + public void functionCall(ITupleReference updateRef, ArrayTupleBuilder cloneUpdateTb, IIndexCursor cursor) + throws HyracksDataException { Object[] tuple = tupleDe.deserializeRecord(updateRef); function.process(tuple); - function.update(updateRef, cloneUpdateTb); + function.update(updateRef, cloneUpdateTb, cursor); } /** @@ -107,11 +109,11 @@ public void functionCall(ITupleReference updateRef, ArrayTupleBuilder cloneUpdat * update pointer * @throws HyracksDataException */ - public void functionCall(ArrayTupleBuilder tb, ITupleReference inPlaceUpdateRef, ArrayTupleBuilder cloneUpdateTb) - throws HyracksDataException { + public void functionCall(ArrayTupleBuilder tb, ITupleReference inPlaceUpdateRef, ArrayTupleBuilder cloneUpdateTb, + IIndexCursor cursor) throws HyracksDataException { Object[] tuple = tupleDe.deserializeRecord(tb, inPlaceUpdateRef); function.process(tuple); - function.update(inPlaceUpdateRef, cloneUpdateTb); + function.update(inPlaceUpdateRef, cloneUpdateTb, cursor); } /** diff --git a/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/StorageType.java b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/StorageType.java new file mode 100644 index 000000000..fb2d1eb63 --- /dev/null +++ b/pregelix/pregelix-dataflow-std/src/main/java/edu/uci/ics/pregelix/dataflow/util/StorageType.java @@ -0,0 +1,21 @@ +/* + * 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.util; + +public enum StorageType { + TreeIndex, + LSMIndex +} diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/FinalAggregateOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/FinalAggregateOperatorDescriptor.java index c0be9dd20..d32cb6b7d 100644 --- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/FinalAggregateOperatorDescriptor.java +++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/FinalAggregateOperatorDescriptor.java @@ -19,6 +19,9 @@ import java.io.DataInputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -58,21 +61,24 @@ public FinalAggregateOperatorDescriptor(JobSpecification spec, IConfigurationFac } @Override + @SuppressWarnings("rawtypes") public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException { return new AbstractUnaryInputSinkOperatorNodePushable() { private Configuration conf = confFactory.createConfiguration(ctx); - @SuppressWarnings("rawtypes") - private GlobalAggregator aggregator = BspUtils.createGlobalAggregator(conf); + private List aggregators = BspUtils.createGlobalAggregators(conf); + private List aggregateClassNames = Arrays.asList(BspUtils.getGlobalAggregatorClassNames(conf)); private FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), inputRdFactory.createRecordDescriptor(ctx)); private ByteBufferInputStream inputStream = new ByteBufferInputStream(); private DataInput input = new DataInputStream(inputStream); - private Writable partialAggregateValue = BspUtils.createFinalAggregateValue(conf); + private List partialAggregateValues = BspUtils.createFinalAggregateValues(conf); @Override public void open() throws HyracksDataException { - aggregator.init(); + for (GlobalAggregator aggregator : aggregators) { + aggregator.init(); + } } @SuppressWarnings("unchecked") @@ -82,11 +88,14 @@ public void nextFrame(ByteBuffer buffer) throws HyracksDataException { int tupleCount = accessor.getTupleCount(); try { for (int i = 0; i < tupleCount; i++) { - int start = accessor.getFieldSlotsLength() + accessor.getTupleStartOffset(i) - + accessor.getFieldStartOffset(i, 0); - inputStream.setByteBuffer(buffer, start); - partialAggregateValue.readFields(input); - aggregator.step(partialAggregateValue); + // iterate over all the aggregators + for (int j = 0; j < partialAggregateValues.size(); j++) { + int start = accessor.getFieldSlotsLength() + accessor.getTupleStartOffset(i) + + accessor.getFieldStartOffset(i, j); + inputStream.setByteBuffer(buffer, start); + partialAggregateValues.get(j).readFields(input); + aggregators.get(j).step(partialAggregateValues.get(j)); + } } } catch (Exception e) { throw new HyracksDataException(e); @@ -102,6 +111,7 @@ public void fail() throws HyracksDataException { @Override public void close() throws HyracksDataException { try { + List aggValues = new ArrayList(); // iterate over hdfs spilled aggregates FileSystem dfs = FileSystem.get(conf); String spillingDir = BspUtils.getGlobalAggregateSpillingDirName(conf, Vertex.getSuperstep()); @@ -111,12 +121,20 @@ public void close() throws HyracksDataException { for (int i = 0; i < files.length; i++) { FileStatus file = files[i]; DataInput dis = dfs.open(file.getPath()); - partialAggregateValue.readFields(dis); - aggregator.step(partialAggregateValue); + for (int j = 0; j < partialAggregateValues.size(); j++) { + GlobalAggregator aggregator = aggregators.get(j); + Writable partialAggregateValue = partialAggregateValues.get(j); + partialAggregateValue.readFields(dis); + aggregator.step(partialAggregateValue); + } } } - Writable finalAggregateValue = aggregator.finishFinal(); - IterationUtils.writeGlobalAggregateValue(conf, jobId, finalAggregateValue); + for (int j = 0; j < partialAggregateValues.size(); j++) { + GlobalAggregator aggregator = aggregators.get(j); + Writable finalAggregateValue = aggregator.finishFinal(); + aggValues.add(finalAggregateValue); + } + IterationUtils.writeGlobalAggregateValue(conf, jobId, aggregateClassNames, aggValues); } catch (IOException e) { throw new HyracksDataException(e); } diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java index b44b64323..a5d2ab73a 100644 --- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java +++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingReadOperatorDescriptor.java @@ -31,11 +31,15 @@ public class MaterializingReadOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor { private static final long serialVersionUID = 1L; private final boolean removeIterationState; + private final String jobId; + private final int iteration; public MaterializingReadOperatorDescriptor(JobSpecification spec, RecordDescriptor recordDescriptor, - boolean removeIterationState) { + boolean removeIterationState, String jobId, int iteration) { super(spec, 1, 1); this.removeIterationState = removeIterationState; + this.jobId = jobId; + this.iteration = iteration - 1; recordDescriptors[0] = recordDescriptor; } @@ -55,8 +59,8 @@ public void open() throws HyracksDataException { @Override public void nextFrame(ByteBuffer buffer) throws HyracksDataException { if (!complete) { - MaterializerTaskState state = (MaterializerTaskState) IterationUtils.getIterationState(ctx, - partition); + MaterializerTaskState state = (MaterializerTaskState) IterationUtils.getIterationState(ctx, jobId, + partition, iteration); RunFileReader in = state.getRunFileWriter().createReader(); writer.open(); try { @@ -85,7 +89,7 @@ public void close() throws HyracksDataException { * remove last iteration's state */ if (removeIterationState) { - IterationUtils.removeIterationState(ctx, partition); + IterationUtils.removeIterationState(ctx, jobId, partition, iteration); } writer.close(); complete = true; diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingWriteOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingWriteOperatorDescriptor.java index 00dcbd107..921dc40e5 100644 --- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingWriteOperatorDescriptor.java +++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/MaterializingWriteOperatorDescriptor.java @@ -38,9 +38,14 @@ public class MaterializingWriteOperatorDescriptor extends AbstractOperatorDescriptor { private static final long serialVersionUID = 1L; private final static int MATERIALIZER_ACTIVITY_ID = 0; + private final String jobId; + private final int iteration; - public MaterializingWriteOperatorDescriptor(JobSpecification spec, RecordDescriptor recordDescriptor) { + public MaterializingWriteOperatorDescriptor(JobSpecification spec, RecordDescriptor recordDescriptor, String jobId, + int iteration) { super(spec, 1, 1); + this.jobId = jobId; + this.iteration = iteration; recordDescriptors[0] = recordDescriptor; } @@ -69,13 +74,12 @@ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, @Override public void open() throws HyracksDataException { /** remove last iteration's state */ - IterationUtils.removeIterationState(ctx, partition); + IterationUtils.removeIterationState(ctx, jobId, partition, iteration); state = new MaterializerTaskState(ctx.getJobletContext().getJobId(), new TaskId(getActivityId(), partition)); INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext(); RuntimeContext context = (RuntimeContext) appContext.getApplicationObject(); - FileReference file = context.createManagedWorkspaceFile(MaterializingWriteOperatorDescriptor.class - .getSimpleName()); + FileReference file = context.createManagedWorkspaceFile(jobId); state.setRunFileWriter(new RunFileWriter(file, ctx.getIOManager())); state.getRunFileWriter().open(); writer.open(); @@ -92,7 +96,7 @@ public void close() throws HyracksDataException { /** * set iteration state */ - IterationUtils.setIterationState(ctx, partition, state); + IterationUtils.setIterationState(ctx, jobId, partition, iteration, state); writer.close(); } diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java index b34879e9b..e16ba48a9 100644 --- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java +++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileScanOperatorDescriptor.java @@ -16,6 +16,8 @@ import java.io.DataOutput; import java.io.IOException; +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; @@ -122,7 +124,7 @@ public void initialize() throws HyracksDataException { @SuppressWarnings("unchecked") private void loadVertices(final IHyracksTaskContext ctx, Configuration conf, int splitId) throws IOException, ClassNotFoundException, InterruptedException, InstantiationException, - IllegalAccessException { + IllegalAccessException, NoSuchFieldException, InvocationTargetException { ByteBuffer frame = ctx.allocateFrame(); FrameTupleAppender appender = new FrameTupleAppender(ctx.getFrameSize()); appender.reset(frame, true); @@ -141,7 +143,11 @@ private void loadVertices(final IHyracksTaskContext ctx, Configuration conf, int /** * set context */ - Vertex.setContext(mapperContext); + ClassLoader cl = ctx.getJobletContext().getClassLoader(); + Class vClass = (Class) cl.loadClass("edu.uci.ics.pregelix.api.graph.Vertex"); + Field contextField = vClass.getDeclaredField("context"); + contextField.setAccessible(true); + contextField.set(null, mapperContext); /** * empty vertex value diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileWriteOperatorDescriptor.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileWriteOperatorDescriptor.java index f3ec40edf..2087ea27d 100644 --- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileWriteOperatorDescriptor.java +++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/VertexFileWriteOperatorDescriptor.java @@ -44,17 +44,20 @@ 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; +import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory; public class VertexFileWriteOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor { private static final long serialVersionUID = 1L; private final IConfigurationFactory confFactory; private final IRecordDescriptorFactory inputRdFactory; + private final IRuntimeHookFactory preHookFactory; public VertexFileWriteOperatorDescriptor(JobSpecification spec, IConfigurationFactory confFactory, - IRecordDescriptorFactory inputRdFactory) { + IRecordDescriptorFactory inputRdFactory, IRuntimeHookFactory preHookFactory) { super(spec, 1, 0); this.confFactory = confFactory; this.inputRdFactory = inputRdFactory; + this.preHookFactory = preHookFactory; } @SuppressWarnings("rawtypes") @@ -85,6 +88,8 @@ public void open() throws HyracksDataException { context = ctxFactory.createContext(conf, partition); context.getConfiguration().setClassLoader(ctx.getJobletContext().getClassLoader()); try { + if (preHookFactory != null) + preHookFactory.createRuntimeHook().configure(ctx); vertexWriter = outputFormat.createVertexWriter(context); } catch (InterruptedException e) { throw new HyracksDataException(e); diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/PJobContext.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/PJobContext.java new file mode 100644 index 000000000..9daed120b --- /dev/null +++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/PJobContext.java @@ -0,0 +1,132 @@ +package edu.uci.ics.pregelix.dataflow.context; + +import java.lang.reflect.Method; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.ConcurrentHashMap; +import java.util.logging.Logger; + +import edu.uci.ics.hyracks.api.context.IHyracksTaskContext; +import edu.uci.ics.hyracks.api.dataflow.state.IStateObject; +import edu.uci.ics.hyracks.api.exceptions.HyracksDataException; +import edu.uci.ics.hyracks.api.io.FileReference; +import edu.uci.ics.pregelix.api.graph.Vertex; + +public class PJobContext { + private static final Logger LOGGER = Logger.getLogger(RuntimeContext.class.getName()); + + private final Map> iterationToFiles = new ConcurrentHashMap>(); + private final Map appStateMap = new ConcurrentHashMap(); + private Long jobIdToSuperStep; + private Boolean jobIdToMove; + + public void close() throws HyracksDataException { + for (Entry> entry : iterationToFiles.entrySet()) + for (FileReference fileRef : entry.getValue()) + fileRef.delete(); + + iterationToFiles.clear(); + appStateMap.clear(); + } + + public void clearState() throws HyracksDataException { + for (Entry> entry : iterationToFiles.entrySet()) + for (FileReference fileRef : entry.getValue()) + fileRef.delete(); + + iterationToFiles.clear(); + appStateMap.clear(); + } + + public Map getAppStateStore() { + return appStateMap; + } + + public static RuntimeContext get(IHyracksTaskContext ctx) { + return (RuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject(); + } + + public void setVertexProperties(long numVertices, long numEdges, long currentIteration, ClassLoader cl) { + if (jobIdToMove == null || jobIdToMove == true) { + if (jobIdToSuperStep == null) { + if (currentIteration <= 0) { + jobIdToSuperStep = 0L; + } else { + jobIdToSuperStep = currentIteration; + } + } + + long superStep = jobIdToSuperStep; + List files = iterationToFiles.remove(superStep - 1); + if (files != null) { + for (FileReference fileRef : files) { + if (fileRef != null) { + fileRef.delete(); + } + } + } + + setProperties(numVertices, numEdges, currentIteration, superStep, false, cl); + } + System.gc(); + } + + public void recoverVertexProperties(long numVertices, long numEdges, long currentIteration, ClassLoader cl) { + if (jobIdToSuperStep == null) { + if (currentIteration <= 0) { + jobIdToSuperStep = 0L; + } else { + jobIdToSuperStep = currentIteration; + } + } + + long superStep = jobIdToSuperStep; + List files = iterationToFiles.remove(superStep - 1); + if (files != null) { + for (FileReference fileRef : files) { + if (fileRef != null) { + fileRef.delete(); + } + } + } + + setProperties(numVertices, numEdges, currentIteration, superStep, true, cl); + } + + public void endSuperStep() { + jobIdToMove = true; + LOGGER.info("end iteration " + Vertex.getSuperstep()); + } + + public Map> getIterationToFiles() { + return iterationToFiles; + } + + private void setProperties(long numVertices, long numEdges, long currentIteration, long superStep, boolean toMove, + ClassLoader cl) { + try { + Class vClass = (Class) cl.loadClass("edu.uci.ics.pregelix.api.graph.Vertex"); + Method superStepMethod = vClass.getMethod("setSuperstep", Long.TYPE); + Method numVerticesMethod = vClass.getMethod("setNumVertices", Long.TYPE); + Method numEdgesMethod = vClass.getMethod("setNumEdges", Long.TYPE); + + if (currentIteration > 0) { + //Vertex.setSuperstep(currentIteration); + superStepMethod.invoke(null, currentIteration); + } else { + //Vertex.setSuperstep(++superStep); + superStepMethod.invoke(null, ++superStep); + } + //Vertex.setNumVertices(numVertices); + numVerticesMethod.invoke(null, numVertices); + //Vertex.setNumEdges(numEdges); + numEdgesMethod.invoke(null, numEdges); + jobIdToSuperStep = superStep; + jobIdToMove = toMove; + LOGGER.info("start iteration " + Vertex.getSuperstep()); + } catch (Exception e) { + throw new IllegalStateException(e); + } + } +} diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/PreDelayPageCleanerPolicy.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/PreDelayPageCleanerPolicy.java index 72adfaef9..aa8336f90 100644 --- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/PreDelayPageCleanerPolicy.java +++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/PreDelayPageCleanerPolicy.java @@ -30,7 +30,7 @@ public void notifyCleanCycleStart(Object monitor) throws HyracksDataException { try { monitor.wait(delay); } catch (InterruptedException e) { - throw new HyracksDataException(e); + //throw new HyracksDataException(e); } } diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java index ab49514e3..af8df72e8 100644 --- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java +++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/RuntimeContext.java @@ -20,7 +20,6 @@ import java.util.Map.Entry; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ThreadFactory; -import java.util.logging.Logger; import edu.uci.ics.hyracks.api.application.INCApplicationContext; import edu.uci.ics.hyracks.api.context.IHyracksTaskContext; @@ -48,8 +47,8 @@ import edu.uci.ics.pregelix.api.graph.Vertex; public class RuntimeContext implements IWorkspaceFileFactory { - private static final Logger LOGGER = Logger.getLogger(RuntimeContext.class.getName()); + private final static int SHUTDOWN_GRACEFUL_PERIOD = 5000; private final IIndexLifecycleManager lcManager; private final ILocalResourceRepository localResourceRepository; private final ResourceIdFactory resourceIdFactory; @@ -57,10 +56,7 @@ public class RuntimeContext implements IWorkspaceFileFactory { private final List vbcs; private final IFileMapManager fileMapManager; private final IOManager ioManager; - private final Map> iterationToFiles = new ConcurrentHashMap>(); - private final Map appStateMap = new ConcurrentHashMap(); - private final Map jobIdToSuperStep = new ConcurrentHashMap(); - private final Map jobIdToMove = new ConcurrentHashMap(); + private final Map activeJobs = new ConcurrentHashMap(); private final ThreadFactory threadFactory = new ThreadFactory() { public Thread newThread(Runnable r) { @@ -91,28 +87,18 @@ public RuntimeContext(INCApplicationContext appCtx) { resourceIdFactory = new ResourceIdFactory(0); } - public void close() throws HyracksDataException { - for (Entry> entry : iterationToFiles.entrySet()) - for (FileReference fileRef : entry.getValue()) - fileRef.delete(); + public synchronized void close() throws HyracksDataException { + for (Entry entry : activeJobs.entrySet()) { + entry.getValue().close(); + } + activeJobs.clear(); + // wait a graceful period until all active operators using tree cursors are dead + try { + wait(SHUTDOWN_GRACEFUL_PERIOD); + } catch (InterruptedException e) { - iterationToFiles.clear(); + } bufferCache.close(); - appStateMap.clear(); - - System.gc(); - } - - public void clearState(String jobId) throws HyracksDataException { - for (Entry> entry : iterationToFiles.entrySet()) - for (FileReference fileRef : entry.getValue()) - fileRef.delete(); - - iterationToFiles.clear(); - appStateMap.clear(); - jobIdToMove.remove(jobId); - jobIdToSuperStep.remove(jobId); - System.gc(); } public ILocalResourceRepository getLocalResourceRepository() { @@ -139,87 +125,55 @@ public IFileMapProvider getFileMapManager() { return fileMapManager; } - public Map getAppStateStore() { - return appStateMap; + public synchronized Map getAppStateStore(String jobId) { + PJobContext activeJob = getActiveJob(jobId); + return activeJob.getAppStateStore(); } public static RuntimeContext get(IHyracksTaskContext ctx) { return (RuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject(); } - 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) { - if (currentIteration <= 0) { - jobIdToSuperStep.put(jobId, 0L); - } else { - jobIdToSuperStep.put(jobId, currentIteration); - } - } - - long superStep = jobIdToSuperStep.get(jobId); - List 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, false); - LOGGER.info("start iteration " + Vertex.getSuperstep()); - } - System.gc(); + public synchronized void setVertexProperties(String jobId, long numVertices, long numEdges, long currentIteration, + ClassLoader cl) { + PJobContext activeJob = getActiveJob(jobId); + activeJob.setVertexProperties(numVertices, numEdges, currentIteration, cl); } 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 currentIteration, ClassLoader cl) { + PJobContext activeJob = getActiveJob(jobId); + activeJob.recoverVertexProperties(numVertices, numEdges, currentIteration, cl); + } - long superStep = jobIdToSuperStep.get(jobId); - List files = iterationToFiles.remove(superStep - 1); - if (files != null) { - for (FileReference fileRef : files) - fileRef.delete(); - } + public synchronized void endSuperStep(String jobId) { + PJobContext activeJob = getActiveJob(jobId); + activeJob.endSuperStep(); + } - 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 clearState(String jobId) throws HyracksDataException { + PJobContext activeJob = getActiveJob(jobId); + activeJob.clearState(); + activeJobs.remove(jobId); } - public synchronized void endSuperStep(String pregelixJobId) { - jobIdToMove.put(pregelixJobId, true); - LOGGER.info("end iteration " + Vertex.getSuperstep()); + private PJobContext getActiveJob(String jobId) { + PJobContext activeJob = activeJobs.get(jobId); + if (activeJob == null) { + activeJob = new PJobContext(); + activeJobs.put(jobId, activeJob); + } + return activeJob; } @Override - public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDataException { - final FileReference fRef = ioManager.createWorkspaceFile(prefix); - List files = iterationToFiles.get(Vertex.getSuperstep()); + public FileReference createManagedWorkspaceFile(String jobId) throws HyracksDataException { + final FileReference fRef = ioManager.createWorkspaceFile(jobId); + PJobContext activeJob = getActiveJob(jobId); + List files = activeJob.getIterationToFiles().get(Vertex.getSuperstep()); if (files == null) { files = new ArrayList(); - iterationToFiles.put(Vertex.getSuperstep(), files); + activeJob.getIterationToFiles().put(Vertex.getSuperstep(), files); } files.add(fRef); return fRef; @@ -229,4 +183,5 @@ public FileReference createManagedWorkspaceFile(String prefix) throws HyracksDat public FileReference createUnmanagedWorkspaceFile(String prefix) throws HyracksDataException { return ioManager.createWorkspaceFile(prefix); } + } \ No newline at end of file diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/StateKey.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/TaskID.java similarity index 64% rename from pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/StateKey.java rename to pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/TaskID.java index cbd90b94b..f219ed2e5 100644 --- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/StateKey.java +++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/TaskID.java @@ -12,34 +12,44 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package edu.uci.ics.pregelix.dataflow.context; -import edu.uci.ics.hyracks.api.job.JobId; +public class TaskID { -public class StateKey { - private final JobId jobId; - private final int partition; + private String jobId; + private int partition; - public StateKey(JobId jobId, int partition) { + public TaskID(String jobId, int partition) { this.jobId = jobId; this.partition = partition; } + public String getJobId() { + return jobId; + } + + public int getPartition() { + return partition; + } + @Override public int hashCode() { - return jobId.hashCode() * partition; + return jobId.hashCode() + partition; } @Override public boolean equals(Object o) { - if (!(o instanceof StateKey)) + if (!(o instanceof TaskID)) { return false; - StateKey key = (StateKey) o; - return key.jobId.equals(jobId) && key.partition == partition; + } + TaskID tid = (TaskID) o; + return jobId.equals(tid.getJobId()) && partition == tid.getPartition(); } @Override public String toString() { - return jobId.toString() + ":" + partition; + return "job:" + jobId + " partition:" + partition; } + } diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/TaskIterationID.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/TaskIterationID.java new file mode 100644 index 000000000..53c6a0c71 --- /dev/null +++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/context/TaskIterationID.java @@ -0,0 +1,51 @@ +package edu.uci.ics.pregelix.dataflow.context; + +public class TaskIterationID { + + private TaskID tid; + private int iteration; + + public TaskIterationID(TaskID tid, int iteration) { + this.tid = tid; + this.iteration = iteration; + } + + public TaskIterationID(String jobId, int partition, int iteration) { + this.tid = new TaskID(jobId, partition); + this.iteration = iteration; + } + + public TaskID getTaskID() { + return tid; + } + + public int getIteration() { + return iteration; + } + + public TaskIterationID getNextTaskIterationID() { + return new TaskIterationID(tid, iteration + 1); + } + + public TaskIterationID getPreviousTaskIterationID() { + return new TaskIterationID(tid, iteration - 1); + } + + @Override + public int hashCode() { + return tid.hashCode() + iteration; + } + + public boolean equals(Object o) { + if (!(o instanceof TaskIterationID)) { + return false; + } + TaskIterationID tiid = (TaskIterationID) o; + return tid.equals(tiid.getTaskID()) && iteration == tiid.getIteration(); + } + + @Override + public String toString() { + return tid.toString() + " iteration:" + iteration; + } +} diff --git a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java index 02097bf09..6de65ca36 100644 --- a/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java +++ b/pregelix/pregelix-dataflow/src/main/java/edu/uci/ics/pregelix/dataflow/util/IterationUtils.java @@ -15,6 +15,7 @@ package edu.uci.ics.pregelix.dataflow.util; import java.io.IOException; +import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; @@ -28,73 +29,66 @@ import edu.uci.ics.hyracks.api.context.IHyracksTaskContext; import edu.uci.ics.hyracks.api.dataflow.state.IStateObject; import edu.uci.ics.hyracks.api.exceptions.HyracksDataException; -import edu.uci.ics.hyracks.api.job.JobId; import edu.uci.ics.pregelix.api.job.PregelixJob; import edu.uci.ics.pregelix.api.util.BspUtils; import edu.uci.ics.pregelix.api.util.JobStateUtils; import edu.uci.ics.pregelix.dataflow.context.RuntimeContext; -import edu.uci.ics.pregelix.dataflow.context.StateKey; +import edu.uci.ics.pregelix.dataflow.context.TaskIterationID; public class IterationUtils { public static final String TMP_DIR = "/tmp/"; - public static void setIterationState(IHyracksTaskContext ctx, int partition, IStateObject state) { + public static void setIterationState(IHyracksTaskContext ctx, String pregelixJobId, int partition, int iteration, + IStateObject state) { INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext(); RuntimeContext context = (RuntimeContext) appContext.getApplicationObject(); - Map map = context.getAppStateStore(); - map.put(new StateKey(ctx.getJobletContext().getJobId(), partition), state); + Map map = context.getAppStateStore(pregelixJobId); + map.put(new TaskIterationID(pregelixJobId, partition, iteration), state); } - public static IStateObject getIterationState(IHyracksTaskContext ctx, int partition) { - JobId currentId = ctx.getJobletContext().getJobId(); - JobId lastId = new JobId(currentId.getId() - 1); + public static IStateObject getIterationState(IHyracksTaskContext ctx, String pregelixJobId, int partition, + int iteration) { INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext(); RuntimeContext context = (RuntimeContext) appContext.getApplicationObject(); - Map map = context.getAppStateStore(); - IStateObject state = map.get(new StateKey(lastId, partition)); - while (state == null) { - /** in case the last job is a checkpointing job */ - lastId = new JobId(lastId.getId() - 1); - state = map.get(new StateKey(lastId, partition)); - } + Map map = context.getAppStateStore(pregelixJobId); + IStateObject state = map.get(new TaskIterationID(pregelixJobId, partition, iteration)); return state; } - public static void removeIterationState(IHyracksTaskContext ctx, int partition) { - JobId currentId = ctx.getJobletContext().getJobId(); - JobId lastId = new JobId(currentId.getId() - 1); + public static void removeIterationState(IHyracksTaskContext ctx, String pregelixJobId, int partition, int iteration) { INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext(); RuntimeContext context = (RuntimeContext) appContext.getApplicationObject(); - Map map = context.getAppStateStore(); - map.remove(new StateKey(lastId, partition)); + Map map = context.getAppStateStore(pregelixJobId); + map.remove(new TaskIterationID(pregelixJobId, partition, iteration)); } - public static void endSuperStep(String giraphJobId, IHyracksTaskContext ctx) { + public static void endSuperStep(String pregelixJobId, IHyracksTaskContext ctx) { INCApplicationContext appContext = ctx.getJobletContext().getApplicationContext(); RuntimeContext context = (RuntimeContext) appContext.getApplicationObject(); - context.endSuperStep(giraphJobId); + context.endSuperStep(pregelixJobId); } - public static void setProperties(String jobId, IHyracksTaskContext ctx, Configuration conf, long currentIteration) { + public static void setProperties(String pregelixJobId, 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); + context.setVertexProperties(pregelixJobId, conf.getLong(PregelixJob.NUM_VERTICE, -1), + conf.getLong(PregelixJob.NUM_EDGES, -1), currentIteration, ctx.getJobletContext().getClassLoader()); } - public static void recoverProperties(String jobId, IHyracksTaskContext ctx, Configuration conf, + public static void recoverProperties(String pregelixJobId, 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); + context.recoverVertexProperties(pregelixJobId, conf.getLong(PregelixJob.NUM_VERTICE, -1), + conf.getLong(PregelixJob.NUM_EDGES, -1), currentIteration, ctx.getJobletContext().getClassLoader()); } - public static void writeTerminationState(Configuration conf, String jobId, boolean terminate) + public static void writeTerminationState(Configuration conf, String pregelixJobId, boolean terminate) throws HyracksDataException { try { FileSystem dfs = FileSystem.get(conf); - String pathStr = IterationUtils.TMP_DIR + jobId; + String pathStr = IterationUtils.TMP_DIR + pregelixJobId; Path path = new Path(pathStr); FSDataOutputStream output = dfs.create(path, true); output.writeBoolean(terminate); @@ -105,14 +99,20 @@ public static void writeTerminationState(Configuration conf, String jobId, boole } } - public static void writeGlobalAggregateValue(Configuration conf, String jobId, Writable agg) - throws HyracksDataException { + public static void writeGlobalAggregateValue(Configuration conf, String pregelixJobId, List aggClassNames, + List aggs) throws HyracksDataException { try { FileSystem dfs = FileSystem.get(conf); - String pathStr = IterationUtils.TMP_DIR + jobId + "agg"; + String pathStr = IterationUtils.TMP_DIR + pregelixJobId + "agg"; Path path = new Path(pathStr); - FSDataOutputStream output = dfs.create(path, true); - agg.write(output); + FSDataOutputStream output; + output = dfs.create(path, true); + for (int i = 0; i < aggs.size(); i++) { + //write agg class name + output.writeUTF(aggClassNames.get(i)); + // write the agg value + aggs.get(i).write(output); + } output.flush(); output.close(); } catch (IOException e) { @@ -120,10 +120,10 @@ public static void writeGlobalAggregateValue(Configuration conf, String jobId, W } } - public static boolean readTerminationState(Configuration conf, String jobId) throws HyracksDataException { + public static boolean readTerminationState(Configuration conf, String pregelixJobId) throws HyracksDataException { try { FileSystem dfs = FileSystem.get(conf); - String pathStr = IterationUtils.TMP_DIR + jobId; + String pathStr = IterationUtils.TMP_DIR + pregelixJobId; Path path = new Path(pathStr); FSDataInputStream input = dfs.open(path); boolean terminate = input.readBoolean(); @@ -134,24 +134,34 @@ public static boolean readTerminationState(Configuration conf, String jobId) thr } } - public static void writeForceTerminationState(Configuration conf, String jobId) throws HyracksDataException { - JobStateUtils.writeForceTerminationState(conf, jobId); + public static void writeForceTerminationState(Configuration conf, String pregelixJobId) throws HyracksDataException { + JobStateUtils.writeForceTerminationState(conf, pregelixJobId); } public static boolean readForceTerminationState(Configuration conf, String jobId) throws HyracksDataException { return JobStateUtils.readForceTerminationState(conf, jobId); } - public static Writable readGlobalAggregateValue(Configuration conf, String jobId) throws HyracksDataException { + public static Writable readGlobalAggregateValue(Configuration conf, String jobId, String aggClassName) + throws HyracksDataException { try { FileSystem dfs = FileSystem.get(conf); String pathStr = IterationUtils.TMP_DIR + jobId + "agg"; Path path = new Path(pathStr); FSDataInputStream input = dfs.open(path); - Writable agg = BspUtils.createFinalAggregateValue(conf); - agg.readFields(input); - input.close(); - return agg; + int numOfAggs = BspUtils.createFinalAggregateValues(conf).size(); + for (int i = 0; i < numOfAggs; i++) { + String aggName = input.readUTF(); + Writable agg = BspUtils.createFinalAggregateValue(conf, aggName); + if (aggName.equals(aggClassName)) { + agg.readFields(input); + input.close(); + return agg; + } else { + agg.readFields(input); + } + } + throw new IllegalStateException("Cannot find the aggregate value for " + aggClassName); } catch (IOException e) { throw new HyracksDataException(e); } diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/PageRankVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/PageRankVertex.java index a866c1c90..2508a1ec6 100644 --- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/PageRankVertex.java +++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/PageRankVertex.java @@ -218,6 +218,7 @@ public static void main(String[] args) throws Exception { job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class); job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class); job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class); + job.setFixedVertexValueSize(true); Client.run(args, job); } diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java index 13c3bf571..c5374c1bf 100644 --- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java +++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/maximalclique/MaximalCliqueVertex.java @@ -277,7 +277,7 @@ public String toString() { private static CliquesWritable readMaximalCliqueResult(Configuration conf) { try { CliquesWritable result = (CliquesWritable) IterationUtils.readGlobalAggregateValue(conf, - BspUtils.getJobId(conf)); + BspUtils.getJobId(conf), MaximalCliqueAggregator.class.getName()); return result; } catch (IOException e) { throw new IllegalStateException(e); @@ -287,7 +287,7 @@ private static CliquesWritable readMaximalCliqueResult(Configuration conf) { public static void main(String[] args) throws Exception { PregelixJob job = new PregelixJob(TriangleCountingVertex.class.getSimpleName()); job.setVertexClass(MaximalCliqueVertex.class); - job.setGlobalAggregatorClass(MaximalCliqueAggregator.class); + job.addGlobalAggregatorClass(MaximalCliqueAggregator.class); job.setDynamicVertexValueSize(true); job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class); job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class); diff --git a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingVertex.java b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingVertex.java index a8d85abd4..cf753bb05 100644 --- a/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingVertex.java +++ b/pregelix/pregelix-example/src/main/java/edu/uci/ics/pregelix/example/trianglecounting/TriangleCountingVertex.java @@ -135,8 +135,8 @@ public VertexWriter createVertexWri private static long readTriangleCountingResult(Configuration conf) { try { - VLongWritable count = (VLongWritable) IterationUtils - .readGlobalAggregateValue(conf, BspUtils.getJobId(conf)); + VLongWritable count = (VLongWritable) IterationUtils.readGlobalAggregateValue(conf, + BspUtils.getJobId(conf), TriangleCountingAggregator.class.getName()); return count.get(); } catch (IOException e) { throw new IllegalStateException(e); @@ -146,7 +146,7 @@ private static long readTriangleCountingResult(Configuration conf) { public static void main(String[] args) throws Exception { PregelixJob job = new PregelixJob(TriangleCountingVertex.class.getSimpleName()); job.setVertexClass(TriangleCountingVertex.class); - job.setGlobalAggregatorClass(TriangleCountingAggregator.class); + job.addGlobalAggregatorClass(TriangleCountingAggregator.class); job.setVertexInputFormatClass(TextTriangleCountingInputFormat.class); job.setVertexOutputFormatClass(TriangleCountingVertexOutputFormat.class); job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class); diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryInnerJoinTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryInnerJoinTest.java index ff1e29f0c..421f2f554 100644 --- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryInnerJoinTest.java +++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryInnerJoinTest.java @@ -56,6 +56,7 @@ public void test() throws Exception { FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH)); job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20); job.setCheckpointHook(ConservativeCheckpointHook.class); + job.setFixedVertexValueSize(true); testCluster.setUp(); Driver driver = new Driver(PageRankVertex.class); diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java index 3fdaf15a8..b3ad11226 100644 --- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java +++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryTest.java @@ -55,6 +55,7 @@ public void test() throws Exception { FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH)); job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20); job.setCheckpointHook(ConservativeCheckpointHook.class); + job.setFixedVertexValueSize(true); testCluster.setUp(); Driver driver = new Driver(PageRankVertex.class); diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryWithoutCheckpointTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryWithoutCheckpointTest.java index e006ccd18..9a2ef2c39 100644 --- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryWithoutCheckpointTest.java +++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/FailureRecoveryWithoutCheckpointTest.java @@ -53,6 +53,7 @@ public void test() throws Exception { FileInputFormat.setInputPaths(job, INPUTPATH); FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH)); job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20); + job.setFixedVertexValueSize(true); testCluster.setUp(); Driver driver = new Driver(PageRankVertex.class); diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/MultiJobConnectedComponentsTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/MultiJobConnectedComponentsTest.java new file mode 100644 index 000000000..65b98450f --- /dev/null +++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/MultiJobConnectedComponentsTest.java @@ -0,0 +1,103 @@ +/* + * 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.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; + +/** + * This test case tests multi-user workload, using ConnectedComponents. + * + * @author yingyib + */ +public class MultiJobConnectedComponentsTest { + private static String INPUTPATH = "data/webmapcomplex"; + private static String OUTPUTPAH = "actual/result"; + private static String OUTPUTPAH2 = "actual/result2"; + private static String EXPECTEDPATH = "src/test/resources/expected/ConnectedComponentsRealComplex"; + + @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(); + Thread thread = new Thread(new Runnable() { + + @Override + public void run() { + try { + Driver driver = new Driver(PageRankVertex.class); + PregelixJob job2 = new PregelixJob(ConnectedComponentsVertex.class.getName()); + job2.setVertexClass(ConnectedComponentsVertex.class); + job2.setVertexClass(ConnectedComponentsVertex.class); + job2.setVertexInputFormatClass(TextConnectedComponentsInputFormat.class); + job2.setVertexOutputFormatClass(SimpleConnectedComponentsVertexOutputFormat.class); + job2.setMessageCombinerClass(ConnectedComponentsVertex.SimpleMinCombiner.class); + job2.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class); + job2.setVertexPartitionerClass(DefaultVertexPartitioner.class); + job2.setDynamicVertexValueSize(true); + FileInputFormat.setInputPaths(job2, INPUTPATH); + FileOutputFormat.setOutputPath(job2, new Path(OUTPUTPAH2)); + job2.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23); + job2.setCheckpointHook(ConservativeCheckpointHook.class); + driver.runJob(job2, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT); + TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH2)); + } catch (Exception e) { + throw new IllegalStateException(e); + } + } + }); + thread.start(); + Driver driver = new Driver(PageRankVertex.class); + driver.runJob(job, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT); + TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH)); + thread.join(); + } catch (Exception e) { + PregelixHyracksIntegrationUtil.deinit(); + testCluster.cleanupHDFS(); + throw e; + } + } +} diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/MultiJobPageRankTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/MultiJobPageRankTest.java new file mode 100644 index 000000000..cfd1b273e --- /dev/null +++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/MultiJobPageRankTest.java @@ -0,0 +1,98 @@ +/* + * 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.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; +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; + +/** + * This test case tests multi-user workload, using PageRank. + * + * @author yingyib + */ +public class MultiJobPageRankTest { + private static String INPUTPATH = "data/webmap"; + private static String OUTPUTPAH = "actual/result"; + private static String OUTPUTPAH2 = "actual/result2"; + private static String EXPECTEDPATH = "src/test/resources/expected/PageRankReal"; + + @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); + job.setFixedVertexValueSize(true); + + testCluster.setUp(); + Thread thread = new Thread(new Runnable() { + + @Override + public void run() { + try { + Driver driver = new Driver(PageRankVertex.class); + PregelixJob job2 = new PregelixJob(PageRankVertex.class.getName()); + job2.setVertexClass(PageRankVertex.class); + job2.setVertexInputFormatClass(TextPageRankInputFormat.class); + job2.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class); + job2.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class); + job2.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class); + FileInputFormat.setInputPaths(job2, INPUTPATH); + FileOutputFormat.setOutputPath(job2, new Path(OUTPUTPAH2)); + job2.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20); + job2.setCheckpointHook(ConservativeCheckpointHook.class); + job2.setFixedVertexValueSize(true); + driver.runJob(job2, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT); + TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH2)); + } catch (Exception e) { + throw new IllegalStateException(e); + } + } + }); + thread.start(); + Driver driver = new Driver(PageRankVertex.class); + driver.runJob(job, "127.0.0.1", PregelixHyracksIntegrationUtil.TEST_HYRACKS_CC_CLIENT_PORT); + TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH)); + thread.join(); + } catch (Exception e) { + PregelixHyracksIntegrationUtil.deinit(); + testCluster.cleanupHDFS(); + throw e; + } + } +} diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/OverflowAggregatorTest.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/OverflowAggregatorTest.java index 474d0a6f7..9badc78d1 100644 --- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/OverflowAggregatorTest.java +++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/OverflowAggregatorTest.java @@ -59,7 +59,7 @@ public void test() throws Exception { FileInputFormat.setInputPaths(job, INPUTPATH); FileOutputFormat.setOutputPath(job, new Path(OUTPUTPAH)); job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20); - job.setGlobalAggregatorClass(OverflowAggregator.class); + job.addGlobalAggregatorClass(OverflowAggregator.class); testCluster.setUp(); Driver driver = new Driver(PageRankVertex.class); @@ -67,7 +67,7 @@ public void test() throws Exception { TestUtils.compareWithResultDir(new File(EXPECTEDPATH), new File(OUTPUTPAH)); Text text = (Text) IterationUtils.readGlobalAggregateValue(job.getConfiguration(), - BspUtils.getJobId(job.getConfiguration())); + BspUtils.getJobId(job.getConfiguration()), OverflowAggregator.class.getName()); Assert.assertEquals(text.getLength(), 20 * 32767); } catch (Exception e) { throw e; diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/UpdateVertex.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/UpdateVertex.java index e2246171d..1e6359d68 100644 --- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/UpdateVertex.java +++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/UpdateVertex.java @@ -15,6 +15,7 @@ package edu.uci.ics.pregelix.example; import java.util.Iterator; +import java.util.Random; import org.apache.hadoop.io.FloatWritable; import org.apache.hadoop.io.Text; @@ -29,6 +30,7 @@ public class UpdateVertex extends Vertex msgIterator) throws Exception { @@ -52,27 +54,51 @@ private void verifyVertexSize(Iterator msgIterator) { int valueSize = getVertexValue().toString().toCharArray().length; long expectedValueSize = msgIterator.next().get(); if (valueSize != expectedValueSize) { - throw new IllegalStateException("vertex id: " + getVertexId() + " vertex value size:" + valueSize - + ", expected value size:" + expectedValueSize); + if (valueSize == -expectedValueSize) { + //verify fixed size update + char[] valueCharArray = getVertexValue().toString().toCharArray(); + for (int i = 0; i < valueCharArray.length; i++) { + if (valueCharArray[i] != 'b') { + throw new IllegalStateException("vertex id: " + getVertexId() + + " has a un-propagated update in the last iteration"); + } + } + } else { + throw new IllegalStateException("vertex id: " + getVertexId() + " vertex value size:" + valueSize + + ", expected value size:" + expectedValueSize); + } + } + if (msgIterator.hasNext()) { + throw new IllegalStateException("more than one message for vertex " + " " + getVertexId() + " " + + getVertexValue()); } } private void updateAndSendMsg() { - int newValueSize = (int) Math.pow(Math.abs(getVertexId().get()), getSuperstep()) % MAX_VALUE_SIZE; + int newValueSize = rand.nextInt(MAX_VALUE_SIZE); char[] charArray = new char[newValueSize]; for (int i = 0; i < charArray.length; i++) { charArray[i] = 'a'; } /** - * set the vertex value + * set a self-message */ - tempValue.set(new String(charArray)); - setVertexValue(tempValue); + msg.set(newValueSize); + boolean fixedSize = getVertexId().get() < 2000; + if (fixedSize) { + int oldSize = getVertexValue().toString().toCharArray().length; + charArray = new char[oldSize]; + for (int i = 0; i < oldSize; i++) { + charArray[i] = 'b'; + } + msg.set(-oldSize); + } /** - * send a self-message + * set the vertex value */ - msg.set(newValueSize); + tempValue.set(new String(charArray)); + setVertexValue(tempValue); sendMsg(getVertexId(), msg); activate(); diff --git a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java index f6857fe20..6ccefd27c 100644 --- a/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java +++ b/pregelix/pregelix-example/src/test/java/edu/uci/ics/pregelix/example/jobgen/JobGenerator.java @@ -74,6 +74,7 @@ private static void generatePageRankJobReal(String jobName, String outputPath) t job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class); job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class); job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class); + job.setFixedVertexValueSize(true); FileInputFormat.setInputPaths(job, HDFS_INPUTPATH); FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH)); job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20); @@ -89,6 +90,7 @@ private static void generatePageRankJobRealComplex(String jobName, String output job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class); job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class); job.setVertexPartitionerClass(DefaultVertexPartitioner.class); + job.setFixedVertexValueSize(true); FileInputFormat.setInputPaths(job, HDFS_INPUTPATH2); FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH2)); job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 23); @@ -116,6 +118,7 @@ private static void generatePageRankJobRealNoCombiner(String jobName, String out job.setVertexInputFormatClass(TextPageRankInputFormat.class); job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class); job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class); + job.setFixedVertexValueSize(true); FileInputFormat.setInputPaths(job, HDFS_INPUTPATH); FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH)); job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20); @@ -190,6 +193,7 @@ private static void generatePageRankJob(String jobName, String outputPath) throw job.setMessageCombinerClass(PageRankVertex.SimpleSumCombiner.class); job.setVertexOutputFormatClass(SimplePageRankVertexOutputFormat.class); job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class); + job.setFixedVertexValueSize(true); FileInputFormat.setInputPaths(job, HDFS_INPUTPATH); FileOutputFormat.setOutputPath(job, new Path(HDFS_OUTPUTPAH)); job.getConfiguration().setLong(PregelixJob.NUM_VERTICE, 20); @@ -227,7 +231,7 @@ private static void generatePageRankJobRealDynamic(String jobName, String output private static void generateTriangleCountingJob(String jobName, String outputPath) throws IOException { PregelixJob job = new PregelixJob(jobName); job.setVertexClass(TriangleCountingVertex.class); - job.setGlobalAggregatorClass(TriangleCountingAggregator.class); + job.addGlobalAggregatorClass(TriangleCountingAggregator.class); job.setVertexInputFormatClass(TextTriangleCountingInputFormat.class); job.setVertexOutputFormatClass(TriangleCountingVertexOutputFormat.class); job.setNoramlizedKeyComputerClass(VLongNormalizedKeyComputer.class); @@ -239,7 +243,7 @@ private static void generateTriangleCountingJob(String jobName, String outputPat private static void generateMaximalCliqueJob(String jobName, String outputPath) throws IOException { PregelixJob job = new PregelixJob(jobName); job.setVertexClass(MaximalCliqueVertex.class); - job.setGlobalAggregatorClass(MaximalCliqueAggregator.class); + job.addGlobalAggregatorClass(MaximalCliqueAggregator.class); job.setDynamicVertexValueSize(true); job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class); job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class); @@ -253,7 +257,7 @@ private static void generateMaximalCliqueJob(String jobName, String outputPath) private static void generateMaximalCliqueJob2(String jobName, String outputPath) throws IOException { PregelixJob job = new PregelixJob(jobName); job.setVertexClass(MaximalCliqueVertex.class); - job.setGlobalAggregatorClass(MaximalCliqueAggregator.class); + job.addGlobalAggregatorClass(MaximalCliqueAggregator.class); job.setDynamicVertexValueSize(true); job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class); job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class); @@ -267,7 +271,7 @@ private static void generateMaximalCliqueJob2(String jobName, String outputPath) private static void generateMaximalCliqueJob3(String jobName, String outputPath) throws IOException { PregelixJob job = new PregelixJob(jobName); job.setVertexClass(MaximalCliqueVertex.class); - job.setGlobalAggregatorClass(MaximalCliqueAggregator.class); + job.addGlobalAggregatorClass(MaximalCliqueAggregator.class); job.setDynamicVertexValueSize(true); job.setVertexInputFormatClass(TextMaximalCliqueInputFormat.class); job.setVertexOutputFormatClass(MaximalCliqueVertexOutputFormat.class); diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml index df72d9bf7..591446cc1 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsReal.xml @@ -125,6 +125,7 @@ mapred.inmem.merge.threshold1000 hadoop.logfile.size10000000 pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextConnectedComponentsInputFormat +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator mapred.job.queue.namedefault mapred.job.tracker.persist.jobstatus.activefalse pregelix.incStateLengthtrue diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml index b0bf02456..32c2a1aa9 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/ConnectedComponentsRealComplex.xml @@ -1,145 +1,146 @@ +fs.s3n.implorg.apache.hadoop.fs.s3native.NativeS3FileSystem +mapred.task.cache.levels2 +hadoop.tmp.dir/tmp/hadoop-${user.name} +hadoop.native.libtrue +map.sort.classorg.apache.hadoop.util.QuickSort +ipc.client.idlethreshold4000 +mapred.system.dir${hadoop.tmp.dir}/mapred/system +mapred.job.tracker.persist.jobstatus.hours0 +io.skip.checksum.errorsfalse +fs.default.namefile:/// +mapred.child.tmp./tmp +fs.har.impl.disable.cachetrue +mapred.skip.reduce.max.skip.groups0 +mapred.jobtracker.instrumentationorg.apache.hadoop.mapred.JobTrackerMetricsInst mapred.tasktracker.dns.nameserverdefault +io.sort.factor10 +mapred.task.timeout600000 +mapred.max.tracker.failures4 +hadoop.rpc.socket.factory.class.defaultorg.apache.hadoop.net.StandardSocketFactory +fs.hdfs.implorg.apache.hadoop.hdfs.DistributedFileSystem mapred.queue.default.acl-administer-jobs* +mapred.queue.default.acl-submit-job* mapred.skip.map.auto.incr.proc.counttrue -mapred.jobtracker.instrumentationorg.apache.hadoop.mapred.JobTrackerMetricsInst -mapred.skip.reduce.auto.incr.proc.counttrue -fs.hsftp.implorg.apache.hadoop.hdfs.HsftpFileSystem -mapred.input.dirfile:/webmapcomplex -mapred.submit.replication10 -ipc.server.tcpnodelayfalse -fs.checkpoint.dir${hadoop.tmp.dir}/dfs/namesecondary -mapred.output.compression.typeRECORD +io.mapfile.bloom.size1048576 +tasktracker.http.threads40 mapred.job.shuffle.merge.percent0.66 -mapred.child.java.opts-Xmx200m -mapred.queue.default.acl-submit-job* -keep.failed.task.filesfalse -mapred.jobtracker.job.history.block.size3145728 +fs.ftp.implorg.apache.hadoop.fs.ftp.FTPFileSystem +pregelix.combinerClassedu.uci.ics.pregelix.example.ConnectedComponentsVertex$SimpleMinCombiner +mapred.output.compressfalse io.bytes.per.checksum512 -mapred.task.tracker.report.address127.0.0.1:0 -hadoop.util.hash.typemurmur -fs.hdfs.implorg.apache.hadoop.hdfs.DistributedFileSystem +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator +topology.node.switch.mapping.implorg.apache.hadoop.net.ScriptBasedMapping +mapred.reduce.slowstart.completed.maps0.05 +mapred.reduce.max.attempts4 fs.ramfs.implorg.apache.hadoop.fs.InMemoryFileSystem -mapred.jobtracker.restart.recoverfalse -fs.hftp.implorg.apache.hadoop.hdfs.HftpFileSystem -fs.checkpoint.period3600 -mapred.child.tmp./tmp -mapred.local.dir.minspacekill0 -map.sort.classorg.apache.hadoop.util.QuickSort -hadoop.logfile.count10 -ipc.client.connection.maxidletime10000 -mapred.output.dir/resultcomplex -io.map.index.skip0 -mapred.tasktracker.expiry.interval600000 -mapred.output.compressfalse -io.seqfile.lazydecompresstrue -mapred.reduce.parallel.copies5 -fs.checkpoint.size67108864 -mapred.job.reduce.input.buffer.percent0.0 -mapred.job.nameConnectedComponents -pregelix.nmkComputerClassedu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer -local.cache.size10737418240 -fs.s3n.implorg.apache.hadoop.fs.s3native.NativeS3FileSystem -mapred.userlog.limit.kb0 +mapred.skip.map.max.skip.records0 +mapred.job.tracker.persist.jobstatus.dir/jobtracker/jobsInfo +fs.s3.buffer.dir${hadoop.tmp.dir}/s3 +job.end.retry.attempts0 fs.file.implorg.apache.hadoop.fs.LocalFileSystem -mapred.task.tracker.http.address0.0.0.0:50060 -mapred.task.timeout600000 -fs.kfs.implorg.apache.hadoop.fs.kfs.KosmosFileSystem +mapred.local.dir.minspacestart0 +mapred.output.compression.typeRECORD +topology.script.number.args100 +io.mapfile.bloom.error.rate0.005 mapred.max.tracker.blacklists4 -fs.s3.buffer.dir${hadoop.tmp.dir}/s3 -mapred.job.tracker.persist.jobstatus.dir/jobtracker/jobsInfo -ipc.client.kill.max10 -mapred.tasktracker.instrumentationorg.apache.hadoop.mapred.TaskTrackerMetricsInst -mapred.reduce.tasks.speculative.executiontrue -io.sort.record.percent0.05 -hadoop.security.authorizationfalse -mapred.max.tracker.failures4 -mapred.jobtracker.taskSchedulerorg.apache.hadoop.mapred.JobQueueTaskScheduler +pregelix.partitionerClassedu.uci.ics.pregelix.api.util.DefaultVertexPartitioner +mapred.task.profile.maps0-2 +mapred.userlog.retain.hours24 pregelix.numVertices23 -mapred.tasktracker.dns.interfacedefault +mapred.job.tracker.persist.jobstatus.activefalse +hadoop.security.authorizationfalse +local.cache.size10737418240 +mapred.min.split.size0 mapred.map.tasks2 -mapred.job.tracker.persist.jobstatus.hours0 -fs.s3.sleepTimeSeconds10 -fs.default.namefile:/// -tasktracker.http.threads40 -mapred.tasktracker.taskmemorymanager.monitoring-interval5000 -hadoop.rpc.socket.factory.class.defaultorg.apache.hadoop.net.StandardSocketFactory +mapred.child.java.opts-Xmx200m +mapred.job.queue.namedefault +ipc.server.listen.queue.size128 +mapred.inmem.merge.threshold1000 +job.end.retry.interval30000 +mapred.skip.attempts.to.start.skipping2 +fs.checkpoint.dir${hadoop.tmp.dir}/dfs/namesecondary mapred.reduce.tasks1 -topology.node.switch.mapping.implorg.apache.hadoop.net.ScriptBasedMapping -pregelix.vertexClassedu.uci.ics.pregelix.example.ConnectedComponentsVertex -mapred.skip.reduce.max.skip.groups0 -io.file.buffer.size4096 +mapred.merge.recordsBeforeProgress10000 +mapred.userlog.limit.kb0 +webinterface.private.actionsfalse +io.sort.spill.percent0.80 +mapred.job.shuffle.input.buffer.percent0.70 +mapred.map.tasks.speculative.executiontrue +mapred.job.nameConnectedComponents +hadoop.util.hash.typemurmur +mapred.map.max.attempts4 +pregelix.incStateLengthtrue +mapred.job.tracker.handler.count10 +mapred.tasktracker.expiry.interval600000 mapred.jobtracker.maxtasks.per.job-1 -mapred.tasktracker.indexcache.mb10 -mapred.tasktracker.map.tasks.maximum2 -fs.har.impl.disable.cachetrue -mapred.task.profile.maps0-2 -hadoop.native.libtrue -fs.s3.block.size67108864 +mapred.jobtracker.job.history.block.size3145728 +keep.failed.task.filesfalse +ipc.client.tcpnodelayfalse +mapred.task.profile.reduces0-2 +mapred.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec +io.map.index.skip0 +ipc.server.tcpnodelayfalse +hadoop.logfile.size10000000 +mapred.reduce.tasks.speculative.executiontrue +fs.checkpoint.period3600 mapred.job.reuse.jvm.num.tasks1 -mapred.job.tracker.http.address0.0.0.0:50030 -mapred.tasktracker.reduce.tasks.maximum2 -io.compression.codecsorg.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec -mapred.job.shuffle.input.buffer.percent0.70 -io.seqfile.compress.blocksize1000000 -mapred.queue.namesdefault -fs.har.implorg.apache.hadoop.fs.HarFileSystem -io.mapfile.bloom.error.rate0.005 -mapred.job.trackerlocal -io.skip.checksum.errorsfalse -mapred.reduce.max.attempts4 +mapred.jobtracker.completeuserjobs.maximum100 fs.s3.maxRetries4 -ipc.server.listen.queue.size128 +mapred.local.dir${hadoop.tmp.dir}/mapred/local +fs.hftp.implorg.apache.hadoop.hdfs.HftpFileSystem fs.trash.interval0 -mapred.local.dir.minspacestart0 -fs.s3.implorg.apache.hadoop.fs.s3.S3FileSystem +fs.s3.sleepTimeSeconds10 +mapred.submit.replication10 +fs.har.implorg.apache.hadoop.fs.HarFileSystem +mapred.map.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec +mapred.tasktracker.dns.interfacedefault +mapred.job.trackerlocal io.seqfile.sorter.recordlimit1000000 -io.mapfile.bloom.size1048576 -io.sort.mb100 -mapred.local.dir${hadoop.tmp.dir}/mapred/local -io.sort.factor10 -mapred.task.profilefalse -job.end.retry.interval30000 +mapred.line.input.format.linespermap1 +mapred.jobtracker.taskSchedulerorg.apache.hadoop.mapred.JobQueueTaskScheduler +mapred.tasktracker.instrumentationorg.apache.hadoop.mapred.TaskTrackerMetricsInst mapred.tasktracker.procfsbasedprocesstree.sleeptime-before-sigkill5000 -mapred.jobtracker.completeuserjobs.maximum100 -mapred.task.profile.reduces0-2 -webinterface.private.actionsfalse -hadoop.tmp.dir/tmp/hadoop-${user.name} -pregelix.combinerClassedu.uci.ics.pregelix.example.ConnectedComponentsVertex$SimpleMinCombiner -mapred.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec -mapred.skip.attempts.to.start.skipping2 +mapred.local.dir.minspacekill0 +io.sort.record.percent0.05 +fs.kfs.implorg.apache.hadoop.fs.kfs.KosmosFileSystem mapred.temp.dir${hadoop.tmp.dir}/mapred/temp -mapred.merge.recordsBeforeProgress10000 -mapred.map.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec -mapred.compress.map.outputfalse -io.sort.spill.percent0.80 +mapred.tasktracker.reduce.tasks.maximum2 fs.checkpoint.edits.dir${fs.checkpoint.dir} -mapred.userlog.retain.hours24 -mapred.system.dir${hadoop.tmp.dir}/mapred/system -mapred.line.input.format.linespermap1 -job.end.retry.attempts0 -ipc.client.idlethreshold4000 -pregelix.vertexOutputFormatClassedu.uci.ics.pregelix.example.ConnectedComponentsVertex$SimpleConnectedComponentsVertexOutputFormat -mapred.reduce.copy.backoff300 -mapred.map.tasks.speculative.executiontrue -pregelix.partitionerClassedu.uci.ics.pregelix.api.util.DefaultVertexPartitioner -mapred.inmem.merge.threshold1000 -hadoop.logfile.size10000000 -pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextConnectedComponentsInputFormat -mapred.job.queue.namedefault -mapred.job.tracker.persist.jobstatus.activefalse -pregelix.incStateLengthtrue -mapred.reduce.slowstart.completed.maps0.05 -topology.script.number.args100 -mapred.skip.map.max.skip.records0 -fs.ftp.implorg.apache.hadoop.fs.ftp.FTPFileSystem -mapred.task.cache.levels2 -mapred.job.tracker.handler.count10 -io.serializationsorg.apache.hadoop.io.serializer.WritableSerialization +mapred.job.reduce.input.buffer.percent0.0 +mapred.tasktracker.indexcache.mb10 +pregelix.nmkComputerClassedu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer +hadoop.logfile.count10 +mapred.skip.reduce.auto.incr.proc.counttrue +io.seqfile.compress.blocksize1000000 +fs.s3.block.size67108864 +mapred.tasktracker.taskmemorymanager.monitoring-interval5000 +mapred.acls.enabledfalse +mapred.queue.namesdefault +fs.hsftp.implorg.apache.hadoop.hdfs.HsftpFileSystem +mapred.task.tracker.http.address0.0.0.0:50060 +pregelix.vertexClassedu.uci.ics.pregelix.example.ConnectedComponentsVertex +mapred.reduce.parallel.copies5 +io.seqfile.lazydecompresstrue +mapred.output.dir/resultcomplex +io.sort.mb100 +ipc.client.connection.maxidletime10000 +mapred.compress.map.outputfalse +mapred.task.tracker.report.address127.0.0.1:0 +ipc.client.kill.max10 ipc.client.connect.max.retries10 -mapred.min.split.size0 -mapred.map.max.attempts4 +fs.s3.implorg.apache.hadoop.fs.s3.S3FileSystem +mapred.job.tracker.http.address0.0.0.0:50030 +mapred.input.dirfile:/webmapcomplex +io.file.buffer.size4096 +mapred.jobtracker.restart.recoverfalse +io.serializationsorg.apache.hadoop.io.serializer.WritableSerialization +pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextConnectedComponentsInputFormat +mapred.reduce.copy.backoff300 +pregelix.vertexOutputFormatClassedu.uci.ics.pregelix.example.ConnectedComponentsVertex$SimpleConnectedComponentsVertexOutputFormat +mapred.task.profilefalse jobclient.output.filterFAILED -ipc.client.tcpnodelayfalse -mapred.acls.enabledfalse +mapred.tasktracker.map.tasks.maximum2 +io.compression.codecsorg.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec +fs.checkpoint.size67108864 \ No newline at end of file diff --git a/pregelix/pregelix-example/src/test/resources/jobs/EarlyTermination.xml b/pregelix/pregelix-example/src/test/resources/jobs/EarlyTermination.xml index d908da85d..d06068d57 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/EarlyTermination.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/EarlyTermination.xml @@ -124,6 +124,7 @@ mapred.inmem.merge.threshold1000 hadoop.logfile.size10000000 pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator mapred.job.queue.namedefault mapred.job.tracker.persist.jobstatus.activefalse mapred.reduce.slowstart.completed.maps0.05 diff --git a/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml b/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml index d5ec8f1c2..01d85a532 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/GraphMutation.xml @@ -124,6 +124,7 @@ mapred.inmem.merge.threshold1000 hadoop.logfile.size10000000 pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator mapred.job.queue.namedefault mapred.job.tracker.persist.jobstatus.activefalse mapred.reduce.slowstart.completed.maps0.05 diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml index b4c42e647..072ea9e25 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique.xml @@ -124,7 +124,7 @@ mapred.inmem.merge.threshold1000 hadoop.logfile.size10000000 pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.maximalclique.TextMaximalCliqueInputFormat -pregelix.aggregatorClassedu.uci.ics.pregelix.example.maximalclique.MaximalCliqueAggregator +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator,class edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueAggregator mapred.job.queue.namedefault mapred.job.tracker.persist.jobstatus.activefalse pregelix.incStateLengthtrue diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml index 6cf075bf4..3ae367dd8 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique2.xml @@ -124,7 +124,7 @@ mapred.inmem.merge.threshold1000 hadoop.logfile.size10000000 pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.maximalclique.TextMaximalCliqueInputFormat -pregelix.aggregatorClassedu.uci.ics.pregelix.example.maximalclique.MaximalCliqueAggregator +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator,class edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueAggregator mapred.job.queue.namedefault mapred.job.tracker.persist.jobstatus.activefalse pregelix.incStateLengthtrue diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml index 49e2e6f9d..6cb617f8d 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/MaximalClique3.xml @@ -125,7 +125,7 @@ mapred.inmem.merge.threshold1000 hadoop.logfile.size10000000 pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.maximalclique.TextMaximalCliqueInputFormat -pregelix.aggregatorClassedu.uci.ics.pregelix.example.maximalclique.MaximalCliqueAggregator +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator,class edu.uci.ics.pregelix.example.maximalclique.MaximalCliqueAggregator mapred.job.queue.namedefault mapred.job.tracker.persist.jobstatus.activefalse pregelix.incStateLengthtrue diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MessageOverflow.xml b/pregelix/pregelix-example/src/test/resources/jobs/MessageOverflow.xml index 8316c64d1..76d6e876d 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/MessageOverflow.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/MessageOverflow.xml @@ -125,6 +125,7 @@ mapred.inmem.merge.threshold1000 hadoop.logfile.size10000000 pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator mapred.job.queue.namedefault mapred.job.tracker.persist.jobstatus.activefalse pregelix.incStateLengthtrue diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MessageOverflowFixedsize.xml b/pregelix/pregelix-example/src/test/resources/jobs/MessageOverflowFixedsize.xml index a894ccdcc..1f5225079 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/MessageOverflowFixedsize.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/MessageOverflowFixedsize.xml @@ -124,6 +124,7 @@ mapred.inmem.merge.threshold1000 hadoop.logfile.size10000000 pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator mapred.job.queue.namedefault mapred.job.tracker.persist.jobstatus.activefalse mapred.reduce.slowstart.completed.maps0.05 diff --git a/pregelix/pregelix-example/src/test/resources/jobs/MessageOverflowLSM.xml b/pregelix/pregelix-example/src/test/resources/jobs/MessageOverflowLSM.xml index a9f8925c7..9d2c9e1ca 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/MessageOverflowLSM.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/MessageOverflowLSM.xml @@ -1,145 +1,146 @@ +fs.s3n.implorg.apache.hadoop.fs.s3native.NativeS3FileSystem +mapred.task.cache.levels2 +hadoop.tmp.dir/tmp/hadoop-${user.name} +hadoop.native.libtrue +map.sort.classorg.apache.hadoop.util.QuickSort +ipc.client.idlethreshold4000 +mapred.system.dir${hadoop.tmp.dir}/mapred/system +mapred.job.tracker.persist.jobstatus.hours0 +io.skip.checksum.errorsfalse +fs.default.namefile:/// +mapred.child.tmp./tmp +fs.har.impl.disable.cachetrue +mapred.skip.reduce.max.skip.groups0 +mapred.jobtracker.instrumentationorg.apache.hadoop.mapred.JobTrackerMetricsInst mapred.tasktracker.dns.nameserverdefault +io.sort.factor10 +pregelix.updateIntensivetrue +mapred.task.timeout600000 +mapred.max.tracker.failures4 +hadoop.rpc.socket.factory.class.defaultorg.apache.hadoop.net.StandardSocketFactory +fs.hdfs.implorg.apache.hadoop.hdfs.DistributedFileSystem mapred.queue.default.acl-administer-jobs* +mapred.queue.default.acl-submit-job* mapred.skip.map.auto.incr.proc.counttrue -mapred.jobtracker.instrumentationorg.apache.hadoop.mapred.JobTrackerMetricsInst -mapred.skip.reduce.auto.incr.proc.counttrue -fs.hsftp.implorg.apache.hadoop.hdfs.HsftpFileSystem -mapred.input.dirfile:/webmap -mapred.submit.replication10 -ipc.server.tcpnodelayfalse -fs.checkpoint.dir${hadoop.tmp.dir}/dfs/namesecondary -mapred.output.compression.typeRECORD +pregelix.framesize2048 +io.mapfile.bloom.size1048576 +tasktracker.http.threads40 mapred.job.shuffle.merge.percent0.66 -mapred.child.java.opts-Xmx200m -mapred.queue.default.acl-submit-job* -keep.failed.task.filesfalse -mapred.jobtracker.job.history.block.size3145728 +fs.ftp.implorg.apache.hadoop.fs.ftp.FTPFileSystem +mapred.output.compressfalse io.bytes.per.checksum512 -mapred.task.tracker.report.address127.0.0.1:0 -hadoop.util.hash.typemurmur -fs.hdfs.implorg.apache.hadoop.hdfs.DistributedFileSystem +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator +topology.node.switch.mapping.implorg.apache.hadoop.net.ScriptBasedMapping +mapred.reduce.slowstart.completed.maps0.05 +mapred.reduce.max.attempts4 fs.ramfs.implorg.apache.hadoop.fs.InMemoryFileSystem -mapred.jobtracker.restart.recoverfalse -fs.hftp.implorg.apache.hadoop.hdfs.HftpFileSystem -fs.checkpoint.period3600 -mapred.child.tmp./tmp -mapred.local.dir.minspacekill0 -map.sort.classorg.apache.hadoop.util.QuickSort -hadoop.logfile.count10 -ipc.client.connection.maxidletime10000 -mapred.output.dir/result -io.map.index.skip0 -mapred.tasktracker.expiry.interval600000 -mapred.output.compressfalse -io.seqfile.lazydecompresstrue -mapred.reduce.parallel.copies5 -fs.checkpoint.size67108864 -mapred.job.reduce.input.buffer.percent0.0 -mapred.job.nameMessage Overflow LSM -pregelix.nmkComputerClassedu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer -local.cache.size10737418240 -fs.s3n.implorg.apache.hadoop.fs.s3native.NativeS3FileSystem -mapred.userlog.limit.kb0 +mapred.skip.map.max.skip.records0 +mapred.job.tracker.persist.jobstatus.dir/jobtracker/jobsInfo +fs.s3.buffer.dir${hadoop.tmp.dir}/s3 +job.end.retry.attempts0 fs.file.implorg.apache.hadoop.fs.LocalFileSystem -mapred.task.tracker.http.address0.0.0.0:50060 -mapred.task.timeout600000 -fs.kfs.implorg.apache.hadoop.fs.kfs.KosmosFileSystem +mapred.local.dir.minspacestart0 +mapred.output.compression.typeRECORD +topology.script.number.args100 +io.mapfile.bloom.error.rate0.005 mapred.max.tracker.blacklists4 -fs.s3.buffer.dir${hadoop.tmp.dir}/s3 -mapred.job.tracker.persist.jobstatus.dir/jobtracker/jobsInfo -ipc.client.kill.max10 -mapred.tasktracker.instrumentationorg.apache.hadoop.mapred.TaskTrackerMetricsInst -mapred.reduce.tasks.speculative.executiontrue -io.sort.record.percent0.05 -hadoop.security.authorizationfalse -mapred.max.tracker.failures4 -mapred.jobtracker.taskSchedulerorg.apache.hadoop.mapred.JobQueueTaskScheduler +mapred.task.profile.maps0-2 +mapred.userlog.retain.hours24 pregelix.numVertices20 -mapred.tasktracker.dns.interfacedefault +mapred.job.tracker.persist.jobstatus.activefalse +hadoop.security.authorizationfalse +local.cache.size10737418240 +mapred.min.split.size0 mapred.map.tasks2 -mapred.job.tracker.persist.jobstatus.hours0 -fs.s3.sleepTimeSeconds10 -fs.default.namefile:/// -tasktracker.http.threads40 -mapred.tasktracker.taskmemorymanager.monitoring-interval5000 -hadoop.rpc.socket.factory.class.defaultorg.apache.hadoop.net.StandardSocketFactory +mapred.child.java.opts-Xmx200m +mapred.job.queue.namedefault +ipc.server.listen.queue.size128 +mapred.inmem.merge.threshold1000 +job.end.retry.interval30000 +mapred.skip.attempts.to.start.skipping2 +fs.checkpoint.dir${hadoop.tmp.dir}/dfs/namesecondary mapred.reduce.tasks1 -topology.node.switch.mapping.implorg.apache.hadoop.net.ScriptBasedMapping -pregelix.vertexClassedu.uci.ics.pregelix.example.MessageOverflowVertex -mapred.skip.reduce.max.skip.groups0 -io.file.buffer.size4096 +mapred.merge.recordsBeforeProgress10000 +mapred.userlog.limit.kb0 +webinterface.private.actionsfalse +io.sort.spill.percent0.80 +mapred.job.shuffle.input.buffer.percent0.70 +mapred.map.tasks.speculative.executiontrue +mapred.job.nameMessage Overflow LSM +hadoop.util.hash.typemurmur +mapred.map.max.attempts4 +pregelix.incStateLengthtrue +mapred.job.tracker.handler.count10 +mapred.tasktracker.expiry.interval600000 mapred.jobtracker.maxtasks.per.job-1 -mapred.tasktracker.indexcache.mb10 -mapred.tasktracker.map.tasks.maximum2 -fs.har.impl.disable.cachetrue -mapred.task.profile.maps0-2 -hadoop.native.libtrue -fs.s3.block.size67108864 +mapred.jobtracker.job.history.block.size3145728 +keep.failed.task.filesfalse +ipc.client.tcpnodelayfalse +mapred.task.profile.reduces0-2 +mapred.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec +io.map.index.skip0 +ipc.server.tcpnodelayfalse +hadoop.logfile.size10000000 +mapred.reduce.tasks.speculative.executiontrue +fs.checkpoint.period3600 mapred.job.reuse.jvm.num.tasks1 -mapred.job.tracker.http.address0.0.0.0:50030 -mapred.tasktracker.reduce.tasks.maximum2 -io.compression.codecsorg.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec -mapred.job.shuffle.input.buffer.percent0.70 -pregelix.updateIntensivetrue -io.seqfile.compress.blocksize1000000 -mapred.queue.namesdefault -fs.har.implorg.apache.hadoop.fs.HarFileSystem -io.mapfile.bloom.error.rate0.005 -mapred.job.trackerlocal -io.skip.checksum.errorsfalse -mapred.reduce.max.attempts4 +mapred.jobtracker.completeuserjobs.maximum100 fs.s3.maxRetries4 -ipc.server.listen.queue.size128 +mapred.local.dir${hadoop.tmp.dir}/mapred/local +fs.hftp.implorg.apache.hadoop.hdfs.HftpFileSystem fs.trash.interval0 -mapred.local.dir.minspacestart0 -fs.s3.implorg.apache.hadoop.fs.s3.S3FileSystem +fs.s3.sleepTimeSeconds10 +mapred.submit.replication10 +fs.har.implorg.apache.hadoop.fs.HarFileSystem +mapred.map.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec +mapred.tasktracker.dns.interfacedefault +mapred.job.trackerlocal io.seqfile.sorter.recordlimit1000000 -io.mapfile.bloom.size1048576 -io.sort.mb100 -mapred.local.dir${hadoop.tmp.dir}/mapred/local -io.sort.factor10 -mapred.task.profilefalse -job.end.retry.interval30000 +mapred.line.input.format.linespermap1 +mapred.jobtracker.taskSchedulerorg.apache.hadoop.mapred.JobQueueTaskScheduler +mapred.tasktracker.instrumentationorg.apache.hadoop.mapred.TaskTrackerMetricsInst mapred.tasktracker.procfsbasedprocesstree.sleeptime-before-sigkill5000 -mapred.jobtracker.completeuserjobs.maximum100 -mapred.task.profile.reduces0-2 -webinterface.private.actionsfalse -hadoop.tmp.dir/tmp/hadoop-${user.name} -pregelix.framesize2048 -mapred.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec -mapred.skip.attempts.to.start.skipping2 +mapred.local.dir.minspacekill0 +io.sort.record.percent0.05 +fs.kfs.implorg.apache.hadoop.fs.kfs.KosmosFileSystem mapred.temp.dir${hadoop.tmp.dir}/mapred/temp -mapred.merge.recordsBeforeProgress10000 -mapred.map.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec -mapred.compress.map.outputfalse -io.sort.spill.percent0.80 +mapred.tasktracker.reduce.tasks.maximum2 fs.checkpoint.edits.dir${fs.checkpoint.dir} -mapred.userlog.retain.hours24 -mapred.system.dir${hadoop.tmp.dir}/mapred/system -mapred.line.input.format.linespermap1 -job.end.retry.attempts0 -ipc.client.idlethreshold4000 -pregelix.vertexOutputFormatClassedu.uci.ics.pregelix.example.MessageOverflowVertex$SimpleMessageOverflowVertexOutputFormat -mapred.reduce.copy.backoff300 -mapred.map.tasks.speculative.executiontrue -mapred.inmem.merge.threshold1000 -hadoop.logfile.size10000000 -pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat -mapred.job.queue.namedefault -mapred.job.tracker.persist.jobstatus.activefalse -pregelix.incStateLengthtrue -mapred.reduce.slowstart.completed.maps0.05 -topology.script.number.args100 -mapred.skip.map.max.skip.records0 -fs.ftp.implorg.apache.hadoop.fs.ftp.FTPFileSystem -mapred.task.cache.levels2 -mapred.job.tracker.handler.count10 -io.serializationsorg.apache.hadoop.io.serializer.WritableSerialization +mapred.job.reduce.input.buffer.percent0.0 +mapred.tasktracker.indexcache.mb10 +pregelix.nmkComputerClassedu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer +hadoop.logfile.count10 +mapred.skip.reduce.auto.incr.proc.counttrue +io.seqfile.compress.blocksize1000000 +fs.s3.block.size67108864 +mapred.tasktracker.taskmemorymanager.monitoring-interval5000 +mapred.acls.enabledfalse +mapred.queue.namesdefault +fs.hsftp.implorg.apache.hadoop.hdfs.HsftpFileSystem +mapred.task.tracker.http.address0.0.0.0:50060 +pregelix.vertexClassedu.uci.ics.pregelix.example.MessageOverflowVertex +mapred.reduce.parallel.copies5 +io.seqfile.lazydecompresstrue +mapred.output.dir/result +io.sort.mb100 +ipc.client.connection.maxidletime10000 +mapred.compress.map.outputfalse +mapred.task.tracker.report.address127.0.0.1:0 +ipc.client.kill.max10 ipc.client.connect.max.retries10 -mapred.min.split.size0 -mapred.map.max.attempts4 +fs.s3.implorg.apache.hadoop.fs.s3.S3FileSystem +mapred.job.tracker.http.address0.0.0.0:50030 +mapred.input.dirfile:/webmap +io.file.buffer.size4096 +mapred.jobtracker.restart.recoverfalse +io.serializationsorg.apache.hadoop.io.serializer.WritableSerialization +pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat +mapred.reduce.copy.backoff300 +pregelix.vertexOutputFormatClassedu.uci.ics.pregelix.example.MessageOverflowVertex$SimpleMessageOverflowVertexOutputFormat +mapred.task.profilefalse jobclient.output.filterFAILED -ipc.client.tcpnodelayfalse -mapred.acls.enabledfalse +mapred.tasktracker.map.tasks.maximum2 +io.compression.codecsorg.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec +fs.checkpoint.size67108864 \ No newline at end of file diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml index 65e0b3085..f1c27ca4b 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRank.xml @@ -125,8 +125,10 @@ mapred.inmem.merge.threshold1000 hadoop.logfile.size10000000 pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.PageRankVertex$SimulatedPageRankVertexInputFormat +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator mapred.job.queue.namedefault mapred.job.tracker.persist.jobstatus.activefalse +pregelix.incStateLengthfalse mapred.reduce.slowstart.completed.maps0.05 topology.script.number.args100 mapred.skip.map.max.skip.records0 diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml index b50b02a45..dfb4e7168 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankReal.xml @@ -1,144 +1,146 @@ +fs.s3n.implorg.apache.hadoop.fs.s3native.NativeS3FileSystem +mapred.task.cache.levels2 +hadoop.tmp.dir/tmp/hadoop-${user.name} +hadoop.native.libtrue +map.sort.classorg.apache.hadoop.util.QuickSort +ipc.client.idlethreshold4000 +mapred.system.dir${hadoop.tmp.dir}/mapred/system +mapred.job.tracker.persist.jobstatus.hours0 +io.skip.checksum.errorsfalse +fs.default.namefile:/// +mapred.child.tmp./tmp +fs.har.impl.disable.cachetrue +mapred.skip.reduce.max.skip.groups0 +mapred.jobtracker.instrumentationorg.apache.hadoop.mapred.JobTrackerMetricsInst mapred.tasktracker.dns.nameserverdefault +io.sort.factor10 +pregelix.checkpointHookedu.uci.ics.pregelix.api.util.ConservativeCheckpointHook +mapred.task.timeout600000 +mapred.max.tracker.failures4 +hadoop.rpc.socket.factory.class.defaultorg.apache.hadoop.net.StandardSocketFactory +fs.hdfs.implorg.apache.hadoop.hdfs.DistributedFileSystem mapred.queue.default.acl-administer-jobs* +mapred.queue.default.acl-submit-job* mapred.skip.map.auto.incr.proc.counttrue -mapred.jobtracker.instrumentationorg.apache.hadoop.mapred.JobTrackerMetricsInst -mapred.skip.reduce.auto.incr.proc.counttrue -fs.hsftp.implorg.apache.hadoop.hdfs.HsftpFileSystem -mapred.input.dirfile:/webmap -mapred.submit.replication10 -ipc.server.tcpnodelayfalse -fs.checkpoint.dir${hadoop.tmp.dir}/dfs/namesecondary -mapred.output.compression.typeRECORD +io.mapfile.bloom.size1048576 +tasktracker.http.threads40 mapred.job.shuffle.merge.percent0.66 -mapred.child.java.opts-Xmx200m -mapred.queue.default.acl-submit-job* -keep.failed.task.filesfalse -mapred.jobtracker.job.history.block.size3145728 +fs.ftp.implorg.apache.hadoop.fs.ftp.FTPFileSystem +pregelix.combinerClassedu.uci.ics.pregelix.example.PageRankVertex$SimpleSumCombiner +mapred.output.compressfalse io.bytes.per.checksum512 -mapred.task.tracker.report.address127.0.0.1:0 -hadoop.util.hash.typemurmur -fs.hdfs.implorg.apache.hadoop.hdfs.DistributedFileSystem +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator +topology.node.switch.mapping.implorg.apache.hadoop.net.ScriptBasedMapping +mapred.reduce.slowstart.completed.maps0.05 +mapred.reduce.max.attempts4 fs.ramfs.implorg.apache.hadoop.fs.InMemoryFileSystem -mapred.jobtracker.restart.recoverfalse -fs.hftp.implorg.apache.hadoop.hdfs.HftpFileSystem -fs.checkpoint.period3600 -mapred.child.tmp./tmp -mapred.local.dir.minspacekill0 -map.sort.classorg.apache.hadoop.util.QuickSort -hadoop.logfile.count10 -ipc.client.connection.maxidletime10000 -mapred.output.dir/result -io.map.index.skip0 -mapred.tasktracker.expiry.interval600000 -mapred.output.compressfalse -io.seqfile.lazydecompresstrue -mapred.reduce.parallel.copies5 -fs.checkpoint.size67108864 -mapred.job.reduce.input.buffer.percent0.0 -mapred.job.namePageRank -pregelix.nmkComputerClassedu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer -local.cache.size10737418240 -fs.s3n.implorg.apache.hadoop.fs.s3native.NativeS3FileSystem -mapred.userlog.limit.kb0 +mapred.skip.map.max.skip.records0 +mapred.job.tracker.persist.jobstatus.dir/jobtracker/jobsInfo +fs.s3.buffer.dir${hadoop.tmp.dir}/s3 +job.end.retry.attempts0 fs.file.implorg.apache.hadoop.fs.LocalFileSystem -mapred.task.tracker.http.address0.0.0.0:50060 -mapred.task.timeout600000 -fs.kfs.implorg.apache.hadoop.fs.kfs.KosmosFileSystem +mapred.local.dir.minspacestart0 +mapred.output.compression.typeRECORD +topology.script.number.args100 +io.mapfile.bloom.error.rate0.005 mapred.max.tracker.blacklists4 -fs.s3.buffer.dir${hadoop.tmp.dir}/s3 -mapred.job.tracker.persist.jobstatus.dir/jobtracker/jobsInfo -ipc.client.kill.max10 -mapred.tasktracker.instrumentationorg.apache.hadoop.mapred.TaskTrackerMetricsInst -mapred.reduce.tasks.speculative.executiontrue -io.sort.record.percent0.05 -hadoop.security.authorizationfalse -mapred.max.tracker.failures4 -mapred.jobtracker.taskSchedulerorg.apache.hadoop.mapred.JobQueueTaskScheduler +mapred.task.profile.maps0-2 +mapred.userlog.retain.hours24 pregelix.numVertices20 -mapred.tasktracker.dns.interfacedefault +mapred.job.tracker.persist.jobstatus.activefalse +hadoop.security.authorizationfalse +local.cache.size10737418240 +mapred.min.split.size0 mapred.map.tasks2 -mapred.job.tracker.persist.jobstatus.hours0 -fs.s3.sleepTimeSeconds10 -fs.default.namefile:/// -tasktracker.http.threads40 -mapred.tasktracker.taskmemorymanager.monitoring-interval5000 -hadoop.rpc.socket.factory.class.defaultorg.apache.hadoop.net.StandardSocketFactory +mapred.child.java.opts-Xmx200m +mapred.job.queue.namedefault +ipc.server.listen.queue.size128 +mapred.inmem.merge.threshold1000 +job.end.retry.interval30000 +mapred.skip.attempts.to.start.skipping2 +fs.checkpoint.dir${hadoop.tmp.dir}/dfs/namesecondary mapred.reduce.tasks1 -topology.node.switch.mapping.implorg.apache.hadoop.net.ScriptBasedMapping -pregelix.vertexClassedu.uci.ics.pregelix.example.PageRankVertex -mapred.skip.reduce.max.skip.groups0 -io.file.buffer.size4096 +mapred.merge.recordsBeforeProgress10000 +mapred.userlog.limit.kb0 +webinterface.private.actionsfalse +io.sort.spill.percent0.80 +mapred.job.shuffle.input.buffer.percent0.70 +mapred.map.tasks.speculative.executiontrue +mapred.job.namePageRank +hadoop.util.hash.typemurmur +mapred.map.max.attempts4 +pregelix.incStateLengthfalse +mapred.job.tracker.handler.count10 +mapred.tasktracker.expiry.interval600000 mapred.jobtracker.maxtasks.per.job-1 -mapred.tasktracker.indexcache.mb10 -mapred.tasktracker.map.tasks.maximum2 -fs.har.impl.disable.cachetrue -mapred.task.profile.maps0-2 -hadoop.native.libtrue -fs.s3.block.size67108864 +mapred.jobtracker.job.history.block.size3145728 +keep.failed.task.filesfalse +ipc.client.tcpnodelayfalse +mapred.task.profile.reduces0-2 +mapred.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec +io.map.index.skip0 +ipc.server.tcpnodelayfalse +hadoop.logfile.size10000000 +mapred.reduce.tasks.speculative.executiontrue +fs.checkpoint.period3600 mapred.job.reuse.jvm.num.tasks1 -mapred.job.tracker.http.address0.0.0.0:50030 -mapred.tasktracker.reduce.tasks.maximum2 -io.compression.codecsorg.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec -mapred.job.shuffle.input.buffer.percent0.70 -io.seqfile.compress.blocksize1000000 -mapred.queue.namesdefault -fs.har.implorg.apache.hadoop.fs.HarFileSystem -pregelix.checkpointHookedu.uci.ics.pregelix.api.util.ConservativeCheckpointHook -io.mapfile.bloom.error.rate0.005 -mapred.job.trackerlocal -io.skip.checksum.errorsfalse -mapred.reduce.max.attempts4 +mapred.jobtracker.completeuserjobs.maximum100 fs.s3.maxRetries4 -ipc.server.listen.queue.size128 +mapred.local.dir${hadoop.tmp.dir}/mapred/local +fs.hftp.implorg.apache.hadoop.hdfs.HftpFileSystem fs.trash.interval0 -mapred.local.dir.minspacestart0 -fs.s3.implorg.apache.hadoop.fs.s3.S3FileSystem +fs.s3.sleepTimeSeconds10 +mapred.submit.replication10 +fs.har.implorg.apache.hadoop.fs.HarFileSystem +mapred.map.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec +mapred.tasktracker.dns.interfacedefault +mapred.job.trackerlocal io.seqfile.sorter.recordlimit1000000 -io.mapfile.bloom.size1048576 -io.sort.mb100 -mapred.local.dir${hadoop.tmp.dir}/mapred/local -io.sort.factor10 -mapred.task.profilefalse -job.end.retry.interval30000 +mapred.line.input.format.linespermap1 +mapred.jobtracker.taskSchedulerorg.apache.hadoop.mapred.JobQueueTaskScheduler +mapred.tasktracker.instrumentationorg.apache.hadoop.mapred.TaskTrackerMetricsInst mapred.tasktracker.procfsbasedprocesstree.sleeptime-before-sigkill5000 -mapred.jobtracker.completeuserjobs.maximum100 -mapred.task.profile.reduces0-2 -webinterface.private.actionsfalse -hadoop.tmp.dir/tmp/hadoop-${user.name} -pregelix.combinerClassedu.uci.ics.pregelix.example.PageRankVertex$SimpleSumCombiner -mapred.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec -mapred.skip.attempts.to.start.skipping2 +mapred.local.dir.minspacekill0 +io.sort.record.percent0.05 +fs.kfs.implorg.apache.hadoop.fs.kfs.KosmosFileSystem mapred.temp.dir${hadoop.tmp.dir}/mapred/temp -mapred.merge.recordsBeforeProgress10000 -mapred.map.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec -mapred.compress.map.outputfalse -io.sort.spill.percent0.80 +mapred.tasktracker.reduce.tasks.maximum2 fs.checkpoint.edits.dir${fs.checkpoint.dir} -mapred.userlog.retain.hours24 -mapred.system.dir${hadoop.tmp.dir}/mapred/system -mapred.line.input.format.linespermap1 -job.end.retry.attempts0 -ipc.client.idlethreshold4000 -pregelix.vertexOutputFormatClassedu.uci.ics.pregelix.example.PageRankVertex$SimplePageRankVertexOutputFormat -mapred.reduce.copy.backoff300 -mapred.map.tasks.speculative.executiontrue -mapred.inmem.merge.threshold1000 -hadoop.logfile.size10000000 -pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat -mapred.job.queue.namedefault -mapred.job.tracker.persist.jobstatus.activefalse -mapred.reduce.slowstart.completed.maps0.05 -topology.script.number.args100 -mapred.skip.map.max.skip.records0 -fs.ftp.implorg.apache.hadoop.fs.ftp.FTPFileSystem -mapred.task.cache.levels2 -mapred.job.tracker.handler.count10 -io.serializationsorg.apache.hadoop.io.serializer.WritableSerialization +mapred.job.reduce.input.buffer.percent0.0 +mapred.tasktracker.indexcache.mb10 +pregelix.nmkComputerClassedu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer +hadoop.logfile.count10 +mapred.skip.reduce.auto.incr.proc.counttrue +io.seqfile.compress.blocksize1000000 +fs.s3.block.size67108864 +mapred.tasktracker.taskmemorymanager.monitoring-interval5000 +mapred.acls.enabledfalse +mapred.queue.namesdefault +fs.hsftp.implorg.apache.hadoop.hdfs.HsftpFileSystem +mapred.task.tracker.http.address0.0.0.0:50060 +pregelix.vertexClassedu.uci.ics.pregelix.example.PageRankVertex +mapred.reduce.parallel.copies5 +io.seqfile.lazydecompresstrue +mapred.output.dir/result +io.sort.mb100 +ipc.client.connection.maxidletime10000 +mapred.compress.map.outputfalse +mapred.task.tracker.report.address127.0.0.1:0 +ipc.client.kill.max10 ipc.client.connect.max.retries10 -mapred.min.split.size0 -mapred.map.max.attempts4 +fs.s3.implorg.apache.hadoop.fs.s3.S3FileSystem +mapred.job.tracker.http.address0.0.0.0:50030 +mapred.input.dirfile:/webmap +io.file.buffer.size4096 +mapred.jobtracker.restart.recoverfalse +io.serializationsorg.apache.hadoop.io.serializer.WritableSerialization +pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat +mapred.reduce.copy.backoff300 +pregelix.vertexOutputFormatClassedu.uci.ics.pregelix.example.PageRankVertex$SimplePageRankVertexOutputFormat +mapred.task.profilefalse jobclient.output.filterFAILED -ipc.client.tcpnodelayfalse -mapred.acls.enabledfalse +mapred.tasktracker.map.tasks.maximum2 +io.compression.codecsorg.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec +fs.checkpoint.size67108864 \ No newline at end of file diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml index 217fbbae6..49a6e201d 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealComplex.xml @@ -1,145 +1,147 @@ +fs.s3n.implorg.apache.hadoop.fs.s3native.NativeS3FileSystem +mapred.task.cache.levels2 +hadoop.tmp.dir/tmp/hadoop-${user.name} +hadoop.native.libtrue +map.sort.classorg.apache.hadoop.util.QuickSort +ipc.client.idlethreshold4000 +mapred.system.dir${hadoop.tmp.dir}/mapred/system +mapred.job.tracker.persist.jobstatus.hours0 +io.skip.checksum.errorsfalse +fs.default.namefile:/// +mapred.child.tmp./tmp +fs.har.impl.disable.cachetrue +mapred.skip.reduce.max.skip.groups0 +mapred.jobtracker.instrumentationorg.apache.hadoop.mapred.JobTrackerMetricsInst mapred.tasktracker.dns.nameserverdefault +io.sort.factor10 +pregelix.checkpointHookedu.uci.ics.pregelix.api.util.ConservativeCheckpointHook +mapred.task.timeout600000 +mapred.max.tracker.failures4 +hadoop.rpc.socket.factory.class.defaultorg.apache.hadoop.net.StandardSocketFactory +fs.hdfs.implorg.apache.hadoop.hdfs.DistributedFileSystem mapred.queue.default.acl-administer-jobs* +mapred.queue.default.acl-submit-job* mapred.skip.map.auto.incr.proc.counttrue -mapred.jobtracker.instrumentationorg.apache.hadoop.mapred.JobTrackerMetricsInst -mapred.skip.reduce.auto.incr.proc.counttrue -fs.hsftp.implorg.apache.hadoop.hdfs.HsftpFileSystem -mapred.input.dirfile:/webmapcomplex -mapred.submit.replication10 -ipc.server.tcpnodelayfalse -fs.checkpoint.dir${hadoop.tmp.dir}/dfs/namesecondary -mapred.output.compression.typeRECORD +io.mapfile.bloom.size1048576 +tasktracker.http.threads40 mapred.job.shuffle.merge.percent0.66 -mapred.child.java.opts-Xmx200m -mapred.queue.default.acl-submit-job* -keep.failed.task.filesfalse -mapred.jobtracker.job.history.block.size3145728 +fs.ftp.implorg.apache.hadoop.fs.ftp.FTPFileSystem +pregelix.combinerClassedu.uci.ics.pregelix.example.PageRankVertex$SimpleSumCombiner +mapred.output.compressfalse io.bytes.per.checksum512 -mapred.task.tracker.report.address127.0.0.1:0 -hadoop.util.hash.typemurmur -fs.hdfs.implorg.apache.hadoop.hdfs.DistributedFileSystem +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator +topology.node.switch.mapping.implorg.apache.hadoop.net.ScriptBasedMapping +mapred.reduce.slowstart.completed.maps0.05 +mapred.reduce.max.attempts4 fs.ramfs.implorg.apache.hadoop.fs.InMemoryFileSystem -mapred.jobtracker.restart.recoverfalse -fs.hftp.implorg.apache.hadoop.hdfs.HftpFileSystem -fs.checkpoint.period3600 -mapred.child.tmp./tmp -mapred.local.dir.minspacekill0 -map.sort.classorg.apache.hadoop.util.QuickSort -hadoop.logfile.count10 -ipc.client.connection.maxidletime10000 -mapred.output.dir/resultcomplex -io.map.index.skip0 -mapred.tasktracker.expiry.interval600000 -mapred.output.compressfalse -io.seqfile.lazydecompresstrue -mapred.reduce.parallel.copies5 -fs.checkpoint.size67108864 -mapred.job.reduce.input.buffer.percent0.0 -mapred.job.namePageRank -pregelix.nmkComputerClassedu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer -local.cache.size10737418240 -fs.s3n.implorg.apache.hadoop.fs.s3native.NativeS3FileSystem -mapred.userlog.limit.kb0 +mapred.skip.map.max.skip.records0 +mapred.job.tracker.persist.jobstatus.dir/jobtracker/jobsInfo +fs.s3.buffer.dir${hadoop.tmp.dir}/s3 +job.end.retry.attempts0 fs.file.implorg.apache.hadoop.fs.LocalFileSystem -mapred.task.tracker.http.address0.0.0.0:50060 -mapred.task.timeout600000 -fs.kfs.implorg.apache.hadoop.fs.kfs.KosmosFileSystem +mapred.local.dir.minspacestart0 +mapred.output.compression.typeRECORD +topology.script.number.args100 +io.mapfile.bloom.error.rate0.005 mapred.max.tracker.blacklists4 -fs.s3.buffer.dir${hadoop.tmp.dir}/s3 -mapred.job.tracker.persist.jobstatus.dir/jobtracker/jobsInfo -ipc.client.kill.max10 -mapred.tasktracker.instrumentationorg.apache.hadoop.mapred.TaskTrackerMetricsInst -mapred.reduce.tasks.speculative.executiontrue -io.sort.record.percent0.05 -hadoop.security.authorizationfalse -mapred.max.tracker.failures4 -mapred.jobtracker.taskSchedulerorg.apache.hadoop.mapred.JobQueueTaskScheduler +pregelix.partitionerClassedu.uci.ics.pregelix.api.util.DefaultVertexPartitioner +mapred.task.profile.maps0-2 +mapred.userlog.retain.hours24 pregelix.numVertices23 -mapred.tasktracker.dns.interfacedefault +mapred.job.tracker.persist.jobstatus.activefalse +hadoop.security.authorizationfalse +local.cache.size10737418240 +mapred.min.split.size0 mapred.map.tasks2 -mapred.job.tracker.persist.jobstatus.hours0 -fs.s3.sleepTimeSeconds10 -fs.default.namefile:/// -tasktracker.http.threads40 -mapred.tasktracker.taskmemorymanager.monitoring-interval5000 -hadoop.rpc.socket.factory.class.defaultorg.apache.hadoop.net.StandardSocketFactory +mapred.child.java.opts-Xmx200m +mapred.job.queue.namedefault +ipc.server.listen.queue.size128 +mapred.inmem.merge.threshold1000 +job.end.retry.interval30000 +mapred.skip.attempts.to.start.skipping2 +fs.checkpoint.dir${hadoop.tmp.dir}/dfs/namesecondary mapred.reduce.tasks1 -topology.node.switch.mapping.implorg.apache.hadoop.net.ScriptBasedMapping -pregelix.vertexClassedu.uci.ics.pregelix.example.PageRankVertex -mapred.skip.reduce.max.skip.groups0 -io.file.buffer.size4096 +mapred.merge.recordsBeforeProgress10000 +mapred.userlog.limit.kb0 +webinterface.private.actionsfalse +io.sort.spill.percent0.80 +mapred.job.shuffle.input.buffer.percent0.70 +mapred.map.tasks.speculative.executiontrue +mapred.job.namePageRank +hadoop.util.hash.typemurmur +mapred.map.max.attempts4 +pregelix.incStateLengthfalse +mapred.job.tracker.handler.count10 +mapred.tasktracker.expiry.interval600000 mapred.jobtracker.maxtasks.per.job-1 -mapred.tasktracker.indexcache.mb10 -mapred.tasktracker.map.tasks.maximum2 -fs.har.impl.disable.cachetrue -mapred.task.profile.maps0-2 -hadoop.native.libtrue -fs.s3.block.size67108864 +mapred.jobtracker.job.history.block.size3145728 +keep.failed.task.filesfalse +ipc.client.tcpnodelayfalse +mapred.task.profile.reduces0-2 +mapred.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec +io.map.index.skip0 +ipc.server.tcpnodelayfalse +hadoop.logfile.size10000000 +mapred.reduce.tasks.speculative.executiontrue +fs.checkpoint.period3600 mapred.job.reuse.jvm.num.tasks1 -mapred.job.tracker.http.address0.0.0.0:50030 -mapred.tasktracker.reduce.tasks.maximum2 -io.compression.codecsorg.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec -mapred.job.shuffle.input.buffer.percent0.70 -io.seqfile.compress.blocksize1000000 -mapred.queue.namesdefault -fs.har.implorg.apache.hadoop.fs.HarFileSystem -pregelix.checkpointHookedu.uci.ics.pregelix.api.util.ConservativeCheckpointHook -io.mapfile.bloom.error.rate0.005 -mapred.job.trackerlocal -io.skip.checksum.errorsfalse -mapred.reduce.max.attempts4 +mapred.jobtracker.completeuserjobs.maximum100 fs.s3.maxRetries4 -ipc.server.listen.queue.size128 +mapred.local.dir${hadoop.tmp.dir}/mapred/local +fs.hftp.implorg.apache.hadoop.hdfs.HftpFileSystem fs.trash.interval0 -mapred.local.dir.minspacestart0 -fs.s3.implorg.apache.hadoop.fs.s3.S3FileSystem +fs.s3.sleepTimeSeconds10 +mapred.submit.replication10 +fs.har.implorg.apache.hadoop.fs.HarFileSystem +mapred.map.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec +mapred.tasktracker.dns.interfacedefault +mapred.job.trackerlocal io.seqfile.sorter.recordlimit1000000 -io.mapfile.bloom.size1048576 -io.sort.mb100 -mapred.local.dir${hadoop.tmp.dir}/mapred/local -io.sort.factor10 -mapred.task.profilefalse -job.end.retry.interval30000 +mapred.line.input.format.linespermap1 +mapred.jobtracker.taskSchedulerorg.apache.hadoop.mapred.JobQueueTaskScheduler +mapred.tasktracker.instrumentationorg.apache.hadoop.mapred.TaskTrackerMetricsInst mapred.tasktracker.procfsbasedprocesstree.sleeptime-before-sigkill5000 -mapred.jobtracker.completeuserjobs.maximum100 -mapred.task.profile.reduces0-2 -webinterface.private.actionsfalse -hadoop.tmp.dir/tmp/hadoop-${user.name} -pregelix.combinerClassedu.uci.ics.pregelix.example.PageRankVertex$SimpleSumCombiner -mapred.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec -mapred.skip.attempts.to.start.skipping2 +mapred.local.dir.minspacekill0 +io.sort.record.percent0.05 +fs.kfs.implorg.apache.hadoop.fs.kfs.KosmosFileSystem mapred.temp.dir${hadoop.tmp.dir}/mapred/temp -mapred.merge.recordsBeforeProgress10000 -mapred.map.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec -mapred.compress.map.outputfalse -io.sort.spill.percent0.80 +mapred.tasktracker.reduce.tasks.maximum2 fs.checkpoint.edits.dir${fs.checkpoint.dir} -mapred.userlog.retain.hours24 -mapred.system.dir${hadoop.tmp.dir}/mapred/system -mapred.line.input.format.linespermap1 -job.end.retry.attempts0 -ipc.client.idlethreshold4000 -pregelix.vertexOutputFormatClassedu.uci.ics.pregelix.example.PageRankVertex$SimplePageRankVertexOutputFormat -mapred.reduce.copy.backoff300 -mapred.map.tasks.speculative.executiontrue -pregelix.partitionerClassedu.uci.ics.pregelix.api.util.DefaultVertexPartitioner -mapred.inmem.merge.threshold1000 -hadoop.logfile.size10000000 -pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat -mapred.job.queue.namedefault -mapred.job.tracker.persist.jobstatus.activefalse -mapred.reduce.slowstart.completed.maps0.05 -topology.script.number.args100 -mapred.skip.map.max.skip.records0 -fs.ftp.implorg.apache.hadoop.fs.ftp.FTPFileSystem -mapred.task.cache.levels2 -mapred.job.tracker.handler.count10 -io.serializationsorg.apache.hadoop.io.serializer.WritableSerialization +mapred.job.reduce.input.buffer.percent0.0 +mapred.tasktracker.indexcache.mb10 +pregelix.nmkComputerClassedu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer +hadoop.logfile.count10 +mapred.skip.reduce.auto.incr.proc.counttrue +io.seqfile.compress.blocksize1000000 +fs.s3.block.size67108864 +mapred.tasktracker.taskmemorymanager.monitoring-interval5000 +mapred.acls.enabledfalse +mapred.queue.namesdefault +fs.hsftp.implorg.apache.hadoop.hdfs.HsftpFileSystem +mapred.task.tracker.http.address0.0.0.0:50060 +pregelix.vertexClassedu.uci.ics.pregelix.example.PageRankVertex +mapred.reduce.parallel.copies5 +io.seqfile.lazydecompresstrue +mapred.output.dir/resultcomplex +io.sort.mb100 +ipc.client.connection.maxidletime10000 +mapred.compress.map.outputfalse +mapred.task.tracker.report.address127.0.0.1:0 +ipc.client.kill.max10 ipc.client.connect.max.retries10 -mapred.min.split.size0 -mapred.map.max.attempts4 +fs.s3.implorg.apache.hadoop.fs.s3.S3FileSystem +mapred.job.tracker.http.address0.0.0.0:50030 +mapred.input.dirfile:/webmapcomplex +io.file.buffer.size4096 +mapred.jobtracker.restart.recoverfalse +io.serializationsorg.apache.hadoop.io.serializer.WritableSerialization +pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat +mapred.reduce.copy.backoff300 +pregelix.vertexOutputFormatClassedu.uci.ics.pregelix.example.PageRankVertex$SimplePageRankVertexOutputFormat +mapred.task.profilefalse jobclient.output.filterFAILED -ipc.client.tcpnodelayfalse -mapred.acls.enabledfalse +mapred.tasktracker.map.tasks.maximum2 +io.compression.codecsorg.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec +fs.checkpoint.size67108864 \ No newline at end of file diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml index c05a4dab8..789ea3260 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealDynamic.xml @@ -125,6 +125,7 @@ mapred.inmem.merge.threshold1000 hadoop.logfile.size10000000 pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator mapred.job.queue.namedefault mapred.job.tracker.persist.jobstatus.activefalse pregelix.incStateLengthtrue diff --git a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml index 636b055bc..796b1d1d9 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/PageRankRealNoCombiner.xml @@ -125,8 +125,10 @@ mapred.inmem.merge.threshold1000 hadoop.logfile.size10000000 pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextPageRankInputFormat +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator mapred.job.queue.namedefault mapred.job.tracker.persist.jobstatus.activefalse +pregelix.incStateLengthfalse mapred.reduce.slowstart.completed.maps0.05 topology.script.number.args100 mapred.skip.map.max.skip.records0 diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml index 225429af1..8834eadf8 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplex.xml @@ -1,145 +1,146 @@ +fs.s3n.implorg.apache.hadoop.fs.s3native.NativeS3FileSystem +mapred.task.cache.levels2 +hadoop.tmp.dir/tmp/hadoop-${user.name} +hadoop.native.libtrue +map.sort.classorg.apache.hadoop.util.QuickSort +ipc.client.idlethreshold4000 +mapred.system.dir${hadoop.tmp.dir}/mapred/system +mapred.job.tracker.persist.jobstatus.hours0 +io.skip.checksum.errorsfalse +fs.default.namefile:/// +mapred.child.tmp./tmp +fs.har.impl.disable.cachetrue +mapred.skip.reduce.max.skip.groups0 +mapred.jobtracker.instrumentationorg.apache.hadoop.mapred.JobTrackerMetricsInst mapred.tasktracker.dns.nameserverdefault +io.sort.factor10 +mapred.task.timeout600000 +mapred.max.tracker.failures4 +hadoop.rpc.socket.factory.class.defaultorg.apache.hadoop.net.StandardSocketFactory +fs.hdfs.implorg.apache.hadoop.hdfs.DistributedFileSystem mapred.queue.default.acl-administer-jobs* +mapred.queue.default.acl-submit-job* mapred.skip.map.auto.incr.proc.counttrue -mapred.jobtracker.instrumentationorg.apache.hadoop.mapred.JobTrackerMetricsInst -mapred.skip.reduce.auto.incr.proc.counttrue -fs.hsftp.implorg.apache.hadoop.hdfs.HsftpFileSystem -mapred.input.dirfile:/webmapcomplex -mapred.submit.replication10 -ipc.server.tcpnodelayfalse -fs.checkpoint.dir${hadoop.tmp.dir}/dfs/namesecondary -mapred.output.compression.typeRECORD +io.mapfile.bloom.size1048576 +tasktracker.http.threads40 mapred.job.shuffle.merge.percent0.66 -mapred.child.java.opts-Xmx200m -mapred.queue.default.acl-submit-job* -keep.failed.task.filesfalse -mapred.jobtracker.job.history.block.size3145728 +fs.ftp.implorg.apache.hadoop.fs.ftp.FTPFileSystem +pregelix.combinerClassedu.uci.ics.pregelix.example.ReachabilityVertex$SimpleReachibilityCombiner +mapred.output.compressfalse io.bytes.per.checksum512 -mapred.task.tracker.report.address127.0.0.1:0 -hadoop.util.hash.typemurmur -fs.hdfs.implorg.apache.hadoop.hdfs.DistributedFileSystem +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator +topology.node.switch.mapping.implorg.apache.hadoop.net.ScriptBasedMapping +mapred.reduce.slowstart.completed.maps0.05 +mapred.reduce.max.attempts4 fs.ramfs.implorg.apache.hadoop.fs.InMemoryFileSystem -mapred.jobtracker.restart.recoverfalse -fs.hftp.implorg.apache.hadoop.hdfs.HftpFileSystem -fs.checkpoint.period3600 -mapred.child.tmp./tmp -mapred.local.dir.minspacekill0 -map.sort.classorg.apache.hadoop.util.QuickSort -hadoop.logfile.count10 -ipc.client.connection.maxidletime10000 -mapred.output.dir/resultcomplex -io.map.index.skip0 -mapred.tasktracker.expiry.interval600000 -mapred.output.compressfalse -io.seqfile.lazydecompresstrue -mapred.reduce.parallel.copies5 -fs.checkpoint.size67108864 -mapred.job.reduce.input.buffer.percent0.0 -mapred.job.nameReachibility -pregelix.nmkComputerClassedu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer -local.cache.size10737418240 -fs.s3n.implorg.apache.hadoop.fs.s3native.NativeS3FileSystem -mapred.userlog.limit.kb0 +mapred.skip.map.max.skip.records0 +mapred.job.tracker.persist.jobstatus.dir/jobtracker/jobsInfo +fs.s3.buffer.dir${hadoop.tmp.dir}/s3 +job.end.retry.attempts0 fs.file.implorg.apache.hadoop.fs.LocalFileSystem -mapred.task.tracker.http.address0.0.0.0:50060 -mapred.task.timeout600000 -fs.kfs.implorg.apache.hadoop.fs.kfs.KosmosFileSystem +mapred.local.dir.minspacestart0 +mapred.output.compression.typeRECORD +topology.script.number.args100 +io.mapfile.bloom.error.rate0.005 mapred.max.tracker.blacklists4 -fs.s3.buffer.dir${hadoop.tmp.dir}/s3 -mapred.job.tracker.persist.jobstatus.dir/jobtracker/jobsInfo -ipc.client.kill.max10 -mapred.tasktracker.instrumentationorg.apache.hadoop.mapred.TaskTrackerMetricsInst -mapred.reduce.tasks.speculative.executiontrue -io.sort.record.percent0.05 -hadoop.security.authorizationfalse -mapred.max.tracker.failures4 -mapred.jobtracker.taskSchedulerorg.apache.hadoop.mapred.JobQueueTaskScheduler +mapred.task.profile.maps0-2 +mapred.userlog.retain.hours24 pregelix.numVertices23 -mapred.tasktracker.dns.interfacedefault +mapred.job.tracker.persist.jobstatus.activefalse +hadoop.security.authorizationfalse +local.cache.size10737418240 +mapred.min.split.size0 mapred.map.tasks2 -mapred.job.tracker.persist.jobstatus.hours0 -fs.s3.sleepTimeSeconds10 -fs.default.namefile:/// -tasktracker.http.threads40 -mapred.tasktracker.taskmemorymanager.monitoring-interval5000 -hadoop.rpc.socket.factory.class.defaultorg.apache.hadoop.net.StandardSocketFactory +mapred.child.java.opts-Xmx200m +mapred.job.queue.namedefault +ipc.server.listen.queue.size128 +mapred.inmem.merge.threshold1000 +job.end.retry.interval30000 +mapred.skip.attempts.to.start.skipping2 +fs.checkpoint.dir${hadoop.tmp.dir}/dfs/namesecondary mapred.reduce.tasks1 -topology.node.switch.mapping.implorg.apache.hadoop.net.ScriptBasedMapping -pregelix.vertexClassedu.uci.ics.pregelix.example.ReachabilityVertex -mapred.skip.reduce.max.skip.groups0 -io.file.buffer.size4096 +mapred.merge.recordsBeforeProgress10000 +mapred.userlog.limit.kb0 +webinterface.private.actionsfalse +io.sort.spill.percent0.80 +mapred.job.shuffle.input.buffer.percent0.70 +mapred.map.tasks.speculative.executiontrue +mapred.job.nameReachibility +hadoop.util.hash.typemurmur +mapred.map.max.attempts4 +mapred.job.tracker.handler.count10 +mapred.tasktracker.expiry.interval600000 mapred.jobtracker.maxtasks.per.job-1 -mapred.tasktracker.indexcache.mb10 -mapred.tasktracker.map.tasks.maximum2 -fs.har.impl.disable.cachetrue -mapred.task.profile.maps0-2 -hadoop.native.libtrue -fs.s3.block.size67108864 +mapred.jobtracker.job.history.block.size3145728 +keep.failed.task.filesfalse +ipc.client.tcpnodelayfalse +mapred.task.profile.reduces0-2 +mapred.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec +io.map.index.skip0 +ipc.server.tcpnodelayfalse +hadoop.logfile.size10000000 +mapred.reduce.tasks.speculative.executiontrue +fs.checkpoint.period3600 mapred.job.reuse.jvm.num.tasks1 -mapred.job.tracker.http.address0.0.0.0:50030 -mapred.tasktracker.reduce.tasks.maximum2 -io.compression.codecsorg.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec -mapred.job.shuffle.input.buffer.percent0.70 -io.seqfile.compress.blocksize1000000 -mapred.queue.namesdefault -fs.har.implorg.apache.hadoop.fs.HarFileSystem -io.mapfile.bloom.error.rate0.005 -mapred.job.trackerlocal -io.skip.checksum.errorsfalse -mapred.reduce.max.attempts4 +ReachibilityVertex.sourceId1 +mapred.jobtracker.completeuserjobs.maximum100 fs.s3.maxRetries4 -ipc.server.listen.queue.size128 -ReachibilityVertex.destId10 +mapred.local.dir${hadoop.tmp.dir}/mapred/local +fs.hftp.implorg.apache.hadoop.hdfs.HftpFileSystem fs.trash.interval0 -mapred.local.dir.minspacestart0 -fs.s3.implorg.apache.hadoop.fs.s3.S3FileSystem +fs.s3.sleepTimeSeconds10 +mapred.submit.replication10 +fs.har.implorg.apache.hadoop.fs.HarFileSystem +mapred.map.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec +mapred.tasktracker.dns.interfacedefault +mapred.job.trackerlocal io.seqfile.sorter.recordlimit1000000 -io.mapfile.bloom.size1048576 -io.sort.mb100 -mapred.local.dir${hadoop.tmp.dir}/mapred/local -io.sort.factor10 -mapred.task.profilefalse -job.end.retry.interval30000 +mapred.line.input.format.linespermap1 +mapred.jobtracker.taskSchedulerorg.apache.hadoop.mapred.JobQueueTaskScheduler +mapred.tasktracker.instrumentationorg.apache.hadoop.mapred.TaskTrackerMetricsInst mapred.tasktracker.procfsbasedprocesstree.sleeptime-before-sigkill5000 -mapred.jobtracker.completeuserjobs.maximum100 -mapred.task.profile.reduces0-2 -webinterface.private.actionsfalse -hadoop.tmp.dir/tmp/hadoop-${user.name} -pregelix.combinerClassedu.uci.ics.pregelix.example.ReachabilityVertex$SimpleReachibilityCombiner -mapred.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec -mapred.skip.attempts.to.start.skipping2 +mapred.local.dir.minspacekill0 +io.sort.record.percent0.05 +fs.kfs.implorg.apache.hadoop.fs.kfs.KosmosFileSystem mapred.temp.dir${hadoop.tmp.dir}/mapred/temp -mapred.merge.recordsBeforeProgress10000 -mapred.map.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec -mapred.compress.map.outputfalse -io.sort.spill.percent0.80 +mapred.tasktracker.reduce.tasks.maximum2 fs.checkpoint.edits.dir${fs.checkpoint.dir} -mapred.userlog.retain.hours24 -mapred.system.dir${hadoop.tmp.dir}/mapred/system -mapred.line.input.format.linespermap1 -job.end.retry.attempts0 -ipc.client.idlethreshold4000 -pregelix.vertexOutputFormatClassedu.uci.ics.pregelix.example.ReachabilityVertex$SimpleReachibilityVertexOutputFormat -mapred.reduce.copy.backoff300 -mapred.map.tasks.speculative.executiontrue -mapred.inmem.merge.threshold1000 -hadoop.logfile.size10000000 -pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextReachibilityVertexInputFormat -mapred.job.queue.namedefault -mapred.job.tracker.persist.jobstatus.activefalse -mapred.reduce.slowstart.completed.maps0.05 -ReachibilityVertex.sourceId1 -topology.script.number.args100 -mapred.skip.map.max.skip.records0 -fs.ftp.implorg.apache.hadoop.fs.ftp.FTPFileSystem -mapred.task.cache.levels2 -mapred.job.tracker.handler.count10 -io.serializationsorg.apache.hadoop.io.serializer.WritableSerialization +mapred.job.reduce.input.buffer.percent0.0 +mapred.tasktracker.indexcache.mb10 +pregelix.nmkComputerClassedu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer +hadoop.logfile.count10 +mapred.skip.reduce.auto.incr.proc.counttrue +io.seqfile.compress.blocksize1000000 +fs.s3.block.size67108864 +mapred.tasktracker.taskmemorymanager.monitoring-interval5000 +mapred.acls.enabledfalse +mapred.queue.namesdefault +fs.hsftp.implorg.apache.hadoop.hdfs.HsftpFileSystem +mapred.task.tracker.http.address0.0.0.0:50060 +pregelix.vertexClassedu.uci.ics.pregelix.example.ReachabilityVertex +mapred.reduce.parallel.copies5 +io.seqfile.lazydecompresstrue +mapred.output.dir/resultcomplex +ReachibilityVertex.destId10 +io.sort.mb100 +ipc.client.connection.maxidletime10000 +mapred.compress.map.outputfalse +mapred.task.tracker.report.address127.0.0.1:0 +ipc.client.kill.max10 ipc.client.connect.max.retries10 -mapred.min.split.size0 -mapred.map.max.attempts4 +fs.s3.implorg.apache.hadoop.fs.s3.S3FileSystem +mapred.job.tracker.http.address0.0.0.0:50030 +mapred.input.dirfile:/webmapcomplex +io.file.buffer.size4096 +mapred.jobtracker.restart.recoverfalse +io.serializationsorg.apache.hadoop.io.serializer.WritableSerialization +pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextReachibilityVertexInputFormat +mapred.reduce.copy.backoff300 +pregelix.vertexOutputFormatClassedu.uci.ics.pregelix.example.ReachabilityVertex$SimpleReachibilityVertexOutputFormat +mapred.task.profilefalse jobclient.output.filterFAILED -ipc.client.tcpnodelayfalse -mapred.acls.enabledfalse +mapred.tasktracker.map.tasks.maximum2 +io.compression.codecsorg.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec +fs.checkpoint.size67108864 \ No newline at end of file diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplexNoConnectivity.xml b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplexNoConnectivity.xml index bd9da928d..234dbf9af 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplexNoConnectivity.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/ReachibilityRealComplexNoConnectivity.xml @@ -1,145 +1,146 @@ +fs.s3n.implorg.apache.hadoop.fs.s3native.NativeS3FileSystem +mapred.task.cache.levels2 +hadoop.tmp.dir/tmp/hadoop-${user.name} +hadoop.native.libtrue +map.sort.classorg.apache.hadoop.util.QuickSort +ipc.client.idlethreshold4000 +mapred.system.dir${hadoop.tmp.dir}/mapred/system +mapred.job.tracker.persist.jobstatus.hours0 +io.skip.checksum.errorsfalse +fs.default.namefile:/// +mapred.child.tmp./tmp +fs.har.impl.disable.cachetrue +mapred.skip.reduce.max.skip.groups0 +mapred.jobtracker.instrumentationorg.apache.hadoop.mapred.JobTrackerMetricsInst mapred.tasktracker.dns.nameserverdefault +io.sort.factor10 +mapred.task.timeout600000 +mapred.max.tracker.failures4 +hadoop.rpc.socket.factory.class.defaultorg.apache.hadoop.net.StandardSocketFactory +fs.hdfs.implorg.apache.hadoop.hdfs.DistributedFileSystem mapred.queue.default.acl-administer-jobs* +mapred.queue.default.acl-submit-job* mapred.skip.map.auto.incr.proc.counttrue -mapred.jobtracker.instrumentationorg.apache.hadoop.mapred.JobTrackerMetricsInst -mapred.skip.reduce.auto.incr.proc.counttrue -fs.hsftp.implorg.apache.hadoop.hdfs.HsftpFileSystem -mapred.input.dirfile:/webmapcomplex -mapred.submit.replication10 -ipc.server.tcpnodelayfalse -fs.checkpoint.dir${hadoop.tmp.dir}/dfs/namesecondary -mapred.output.compression.typeRECORD +io.mapfile.bloom.size1048576 +tasktracker.http.threads40 mapred.job.shuffle.merge.percent0.66 -mapred.child.java.opts-Xmx200m -mapred.queue.default.acl-submit-job* -keep.failed.task.filesfalse -mapred.jobtracker.job.history.block.size3145728 +fs.ftp.implorg.apache.hadoop.fs.ftp.FTPFileSystem +pregelix.combinerClassedu.uci.ics.pregelix.example.ReachabilityVertex$SimpleReachibilityCombiner +mapred.output.compressfalse io.bytes.per.checksum512 -mapred.task.tracker.report.address127.0.0.1:0 -hadoop.util.hash.typemurmur -fs.hdfs.implorg.apache.hadoop.hdfs.DistributedFileSystem +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator +topology.node.switch.mapping.implorg.apache.hadoop.net.ScriptBasedMapping +mapred.reduce.slowstart.completed.maps0.05 +mapred.reduce.max.attempts4 fs.ramfs.implorg.apache.hadoop.fs.InMemoryFileSystem -mapred.jobtracker.restart.recoverfalse -fs.hftp.implorg.apache.hadoop.hdfs.HftpFileSystem -fs.checkpoint.period3600 -mapred.child.tmp./tmp -mapred.local.dir.minspacekill0 -map.sort.classorg.apache.hadoop.util.QuickSort -hadoop.logfile.count10 -ipc.client.connection.maxidletime10000 -mapred.output.dir/resultcomplex -io.map.index.skip0 -mapred.tasktracker.expiry.interval600000 -mapred.output.compressfalse -io.seqfile.lazydecompresstrue -mapred.reduce.parallel.copies5 -fs.checkpoint.size67108864 -mapred.job.reduce.input.buffer.percent0.0 -mapred.job.nameReachibility -pregelix.nmkComputerClassedu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer -local.cache.size10737418240 -fs.s3n.implorg.apache.hadoop.fs.s3native.NativeS3FileSystem -mapred.userlog.limit.kb0 +mapred.skip.map.max.skip.records0 +mapred.job.tracker.persist.jobstatus.dir/jobtracker/jobsInfo +fs.s3.buffer.dir${hadoop.tmp.dir}/s3 +job.end.retry.attempts0 fs.file.implorg.apache.hadoop.fs.LocalFileSystem -mapred.task.tracker.http.address0.0.0.0:50060 -mapred.task.timeout600000 -fs.kfs.implorg.apache.hadoop.fs.kfs.KosmosFileSystem +mapred.local.dir.minspacestart0 +mapred.output.compression.typeRECORD +topology.script.number.args100 +io.mapfile.bloom.error.rate0.005 mapred.max.tracker.blacklists4 -fs.s3.buffer.dir${hadoop.tmp.dir}/s3 -mapred.job.tracker.persist.jobstatus.dir/jobtracker/jobsInfo -ipc.client.kill.max10 -mapred.tasktracker.instrumentationorg.apache.hadoop.mapred.TaskTrackerMetricsInst -mapred.reduce.tasks.speculative.executiontrue -io.sort.record.percent0.05 -hadoop.security.authorizationfalse -mapred.max.tracker.failures4 -mapred.jobtracker.taskSchedulerorg.apache.hadoop.mapred.JobQueueTaskScheduler +mapred.task.profile.maps0-2 +mapred.userlog.retain.hours24 pregelix.numVertices23 -mapred.tasktracker.dns.interfacedefault +mapred.job.tracker.persist.jobstatus.activefalse +hadoop.security.authorizationfalse +local.cache.size10737418240 +mapred.min.split.size0 mapred.map.tasks2 -mapred.job.tracker.persist.jobstatus.hours0 -fs.s3.sleepTimeSeconds10 -fs.default.namefile:/// -tasktracker.http.threads40 -mapred.tasktracker.taskmemorymanager.monitoring-interval5000 -hadoop.rpc.socket.factory.class.defaultorg.apache.hadoop.net.StandardSocketFactory +mapred.child.java.opts-Xmx200m +mapred.job.queue.namedefault +ipc.server.listen.queue.size128 +mapred.inmem.merge.threshold1000 +job.end.retry.interval30000 +mapred.skip.attempts.to.start.skipping2 +fs.checkpoint.dir${hadoop.tmp.dir}/dfs/namesecondary mapred.reduce.tasks1 -topology.node.switch.mapping.implorg.apache.hadoop.net.ScriptBasedMapping -pregelix.vertexClassedu.uci.ics.pregelix.example.ReachabilityVertex -mapred.skip.reduce.max.skip.groups0 -io.file.buffer.size4096 +mapred.merge.recordsBeforeProgress10000 +mapred.userlog.limit.kb0 +webinterface.private.actionsfalse +io.sort.spill.percent0.80 +mapred.job.shuffle.input.buffer.percent0.70 +mapred.map.tasks.speculative.executiontrue +mapred.job.nameReachibility +hadoop.util.hash.typemurmur +mapred.map.max.attempts4 +mapred.job.tracker.handler.count10 +mapred.tasktracker.expiry.interval600000 mapred.jobtracker.maxtasks.per.job-1 -mapred.tasktracker.indexcache.mb10 -mapred.tasktracker.map.tasks.maximum2 -fs.har.impl.disable.cachetrue -mapred.task.profile.maps0-2 -hadoop.native.libtrue -fs.s3.block.size67108864 +mapred.jobtracker.job.history.block.size3145728 +keep.failed.task.filesfalse +ipc.client.tcpnodelayfalse +mapred.task.profile.reduces0-2 +mapred.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec +io.map.index.skip0 +ipc.server.tcpnodelayfalse +hadoop.logfile.size10000000 +mapred.reduce.tasks.speculative.executiontrue +fs.checkpoint.period3600 mapred.job.reuse.jvm.num.tasks1 -mapred.job.tracker.http.address0.0.0.0:50030 -mapred.tasktracker.reduce.tasks.maximum2 -io.compression.codecsorg.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec -mapred.job.shuffle.input.buffer.percent0.70 -io.seqfile.compress.blocksize1000000 -mapred.queue.namesdefault -fs.har.implorg.apache.hadoop.fs.HarFileSystem -io.mapfile.bloom.error.rate0.005 -mapred.job.trackerlocal -io.skip.checksum.errorsfalse -mapred.reduce.max.attempts4 +ReachibilityVertex.sourceId1 +mapred.jobtracker.completeuserjobs.maximum100 fs.s3.maxRetries4 -ipc.server.listen.queue.size128 -ReachibilityVertex.destId25 +mapred.local.dir${hadoop.tmp.dir}/mapred/local +fs.hftp.implorg.apache.hadoop.hdfs.HftpFileSystem fs.trash.interval0 -mapred.local.dir.minspacestart0 -fs.s3.implorg.apache.hadoop.fs.s3.S3FileSystem +fs.s3.sleepTimeSeconds10 +mapred.submit.replication10 +fs.har.implorg.apache.hadoop.fs.HarFileSystem +mapred.map.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec +mapred.tasktracker.dns.interfacedefault +mapred.job.trackerlocal io.seqfile.sorter.recordlimit1000000 -io.mapfile.bloom.size1048576 -io.sort.mb100 -mapred.local.dir${hadoop.tmp.dir}/mapred/local -io.sort.factor10 -mapred.task.profilefalse -job.end.retry.interval30000 +mapred.line.input.format.linespermap1 +mapred.jobtracker.taskSchedulerorg.apache.hadoop.mapred.JobQueueTaskScheduler +mapred.tasktracker.instrumentationorg.apache.hadoop.mapred.TaskTrackerMetricsInst mapred.tasktracker.procfsbasedprocesstree.sleeptime-before-sigkill5000 -mapred.jobtracker.completeuserjobs.maximum100 -mapred.task.profile.reduces0-2 -webinterface.private.actionsfalse -hadoop.tmp.dir/tmp/hadoop-${user.name} -pregelix.combinerClassedu.uci.ics.pregelix.example.ReachabilityVertex$SimpleReachibilityCombiner -mapred.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec -mapred.skip.attempts.to.start.skipping2 +mapred.local.dir.minspacekill0 +io.sort.record.percent0.05 +fs.kfs.implorg.apache.hadoop.fs.kfs.KosmosFileSystem mapred.temp.dir${hadoop.tmp.dir}/mapred/temp -mapred.merge.recordsBeforeProgress10000 -mapred.map.output.compression.codecorg.apache.hadoop.io.compress.DefaultCodec -mapred.compress.map.outputfalse -io.sort.spill.percent0.80 +mapred.tasktracker.reduce.tasks.maximum2 fs.checkpoint.edits.dir${fs.checkpoint.dir} -mapred.userlog.retain.hours24 -mapred.system.dir${hadoop.tmp.dir}/mapred/system -mapred.line.input.format.linespermap1 -job.end.retry.attempts0 -ipc.client.idlethreshold4000 -pregelix.vertexOutputFormatClassedu.uci.ics.pregelix.example.ReachabilityVertex$SimpleReachibilityVertexOutputFormat -mapred.reduce.copy.backoff300 -mapred.map.tasks.speculative.executiontrue -mapred.inmem.merge.threshold1000 -hadoop.logfile.size10000000 -pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextReachibilityVertexInputFormat -mapred.job.queue.namedefault -mapred.job.tracker.persist.jobstatus.activefalse -mapred.reduce.slowstart.completed.maps0.05 -ReachibilityVertex.sourceId1 -topology.script.number.args100 -mapred.skip.map.max.skip.records0 -fs.ftp.implorg.apache.hadoop.fs.ftp.FTPFileSystem -mapred.task.cache.levels2 -mapred.job.tracker.handler.count10 -io.serializationsorg.apache.hadoop.io.serializer.WritableSerialization +mapred.job.reduce.input.buffer.percent0.0 +mapred.tasktracker.indexcache.mb10 +pregelix.nmkComputerClassedu.uci.ics.pregelix.example.data.VLongNormalizedKeyComputer +hadoop.logfile.count10 +mapred.skip.reduce.auto.incr.proc.counttrue +io.seqfile.compress.blocksize1000000 +fs.s3.block.size67108864 +mapred.tasktracker.taskmemorymanager.monitoring-interval5000 +mapred.acls.enabledfalse +mapred.queue.namesdefault +fs.hsftp.implorg.apache.hadoop.hdfs.HsftpFileSystem +mapred.task.tracker.http.address0.0.0.0:50060 +pregelix.vertexClassedu.uci.ics.pregelix.example.ReachabilityVertex +mapred.reduce.parallel.copies5 +io.seqfile.lazydecompresstrue +mapred.output.dir/resultcomplex +ReachibilityVertex.destId25 +io.sort.mb100 +ipc.client.connection.maxidletime10000 +mapred.compress.map.outputfalse +mapred.task.tracker.report.address127.0.0.1:0 +ipc.client.kill.max10 ipc.client.connect.max.retries10 -mapred.min.split.size0 -mapred.map.max.attempts4 +fs.s3.implorg.apache.hadoop.fs.s3.S3FileSystem +mapred.job.tracker.http.address0.0.0.0:50030 +mapred.input.dirfile:/webmapcomplex +io.file.buffer.size4096 +mapred.jobtracker.restart.recoverfalse +io.serializationsorg.apache.hadoop.io.serializer.WritableSerialization +pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextReachibilityVertexInputFormat +mapred.reduce.copy.backoff300 +pregelix.vertexOutputFormatClassedu.uci.ics.pregelix.example.ReachabilityVertex$SimpleReachibilityVertexOutputFormat +mapred.task.profilefalse jobclient.output.filterFAILED -ipc.client.tcpnodelayfalse -mapred.acls.enabledfalse +mapred.tasktracker.map.tasks.maximum2 +io.compression.codecsorg.apache.hadoop.io.compress.DefaultCodec,org.apache.hadoop.io.compress.GzipCodec,org.apache.hadoop.io.compress.BZip2Codec +fs.checkpoint.size67108864 \ No newline at end of file diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml index 9acd7bc60..b1c57c52b 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPaths.xml @@ -126,6 +126,7 @@ mapred.inmem.merge.threshold1000 hadoop.logfile.size10000000 pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.PageRankVertex$SimulatedPageRankVertexInputFormat +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator mapred.job.queue.namedefault mapred.job.tracker.persist.jobstatus.activefalse mapred.reduce.slowstart.completed.maps0.05 diff --git a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml index 6c255752a..f1d2dc620 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/ShortestPathsReal.xml @@ -126,6 +126,7 @@ mapred.inmem.merge.threshold1000 hadoop.logfile.size10000000 pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.inputformat.TextShortestPathsInputFormat +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator mapred.job.queue.namedefault mapred.job.tracker.persist.jobstatus.activefalse mapred.reduce.slowstart.completed.maps0.05 diff --git a/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml b/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml index 4a40a6a2f..951ac6f19 100644 --- a/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml +++ b/pregelix/pregelix-example/src/test/resources/jobs/TriangleCounting.xml @@ -123,7 +123,7 @@ mapred.inmem.merge.threshold1000 hadoop.logfile.size10000000 pregelix.vertexInputFormatClassedu.uci.ics.pregelix.example.trianglecounting.TextTriangleCountingInputFormat -pregelix.aggregatorClassedu.uci.ics.pregelix.example.trianglecounting.TriangleCountingAggregator +pregelix.aggregatorClassclass edu.uci.ics.pregelix.api.util.GlobalCountAggregator,class edu.uci.ics.pregelix.example.trianglecounting.TriangleCountingAggregator mapred.job.queue.namedefault mapred.job.tracker.persist.jobstatus.activefalse mapred.reduce.slowstart.completed.maps0.05 diff --git a/pregelix/pregelix-runtime/pom.xml b/pregelix/pregelix-runtime/pom.xml index 6564eb0ba..245c0f583 100644 --- a/pregelix/pregelix-runtime/pom.xml +++ b/pregelix/pregelix-runtime/pom.xml @@ -139,6 +139,13 @@ jar compile + + edu.uci.ics.hyracks + hyracks-storage-common + 0.2.10-SNAPSHOT + jar + compile + edu.uci.ics.hyracks hyracks-storage-am-btree diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/NCApplicationEntryPoint.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/NCApplicationEntryPoint.java index 9607494fe..fe72d7a5c 100644 --- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/NCApplicationEntryPoint.java +++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/bootstrap/NCApplicationEntryPoint.java @@ -19,9 +19,11 @@ import edu.uci.ics.pregelix.dataflow.context.RuntimeContext; public class NCApplicationEntryPoint implements INCApplicationEntryPoint { + private RuntimeContext rCtx = null; + @Override public void start(INCApplicationContext ncAppCtx, String[] args) throws Exception { - RuntimeContext rCtx = new RuntimeContext(ncAppCtx); + rCtx = new RuntimeContext(ncAppCtx); ncAppCtx.setApplicationObject(rCtx); } @@ -32,6 +34,6 @@ public void notifyStartupComplete() throws Exception { @Override public void stop() throws Exception { - + rCtx.close(); } } \ No newline at end of file diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java index f3a0bb4a8..3e4a81106 100644 --- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java +++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/ComputeUpdateFunctionFactory.java @@ -31,6 +31,8 @@ import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder; import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender; import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference; +import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor; +import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor; import edu.uci.ics.pregelix.api.graph.GlobalAggregator; import edu.uci.ics.pregelix.api.graph.MsgList; import edu.uci.ics.pregelix.api.graph.Vertex; @@ -58,9 +60,9 @@ public IUpdateFunction createFunction() { private final ArrayTupleBuilder tbMsg = new ArrayTupleBuilder(2); private final ArrayTupleBuilder tbAlive = new ArrayTupleBuilder(2); private final ArrayTupleBuilder tbTerminate = new ArrayTupleBuilder(1); - private final ArrayTupleBuilder tbGlobalAggregate = new ArrayTupleBuilder(1); private final ArrayTupleBuilder tbInsert = new ArrayTupleBuilder(2); private final ArrayTupleBuilder tbDelete = new ArrayTupleBuilder(1); + private ArrayTupleBuilder tbGlobalAggregate; // for writing out to message channel private IFrameWriter writerMsg; @@ -83,7 +85,7 @@ public IUpdateFunction createFunction() { private IFrameWriter writerGlobalAggregate; private FrameTupleAppender appenderGlobalAggregate; private ByteBuffer bufferGlobalAggregate; - private GlobalAggregator aggregator; + private List aggregators; // for writing out to insert vertex channel private IFrameWriter writerInsert; @@ -110,9 +112,14 @@ public IUpdateFunction createFunction() { public void open(IHyracksTaskContext ctx, RecordDescriptor rd, IFrameWriter... writers) throws HyracksDataException { this.conf = confFactory.createConfiguration(ctx); - this.dynamicStateLength = BspUtils.getDynamicVertexValueSize(conf); - this.aggregator = BspUtils.createGlobalAggregator(conf); - this.aggregator.init(); + //LSM index does not have in-place update + this.dynamicStateLength = BspUtils.getDynamicVertexValueSize(conf) || BspUtils.useLSM(conf); + this.aggregators = BspUtils.createGlobalAggregators(conf); + for (int i = 0; i < aggregators.size(); i++) { + this.aggregators.get(i).init(); + } + + this.tbGlobalAggregate = new ArrayTupleBuilder(aggregators.size()); this.writerMsg = writers[0]; this.bufferMsg = ctx.allocateFrame(); @@ -197,6 +204,10 @@ public void process(Object[] tuple) throws HyracksDataException { } vertex.finishCompute(); } catch (Exception e) { + ClassLoader cl1 = vertex.getClass().getClassLoader(); + ClassLoader cl2 = msgContentList.get(0).getClass().getClassLoader(); + System.out.println("cl1 " + cl1); + System.out.println("cl2 " + cl2); throw new HyracksDataException(e); } @@ -208,7 +219,9 @@ public void process(Object[] tuple) throws HyracksDataException { if (msgContentList.segmentEnd()) { /** the if condition makes sure aggregate only calls once per-vertex */ - aggregator.step(vertex); + for (int i = 0; i < aggregators.size(); i++) { + aggregators.get(i).step(vertex); + } } } @@ -230,13 +243,15 @@ public void close() throws HyracksDataException { private void writeOutGlobalAggregate() throws HyracksDataException { try { - /** - * get partial aggregate result and flush to the final - * aggregator - */ - Writable agg = aggregator.finishPartial(); - agg.write(tbGlobalAggregate.getDataOutput()); - tbGlobalAggregate.addFieldEndOffset(); + for (int i = 0; i < aggregators.size(); i++) { + /** + * get partial aggregate result and flush to the final + * aggregator + */ + Writable agg = aggregators.get(i).finishPartial(); + agg.write(tbGlobalAggregate.getDataOutput()); + tbGlobalAggregate.addFieldEndOffset(); + } if (!appenderGlobalAggregate.append(tbGlobalAggregate.getFieldEndOffsets(), tbGlobalAggregate.getByteArray(), 0, tbGlobalAggregate.getSize())) { // aggregate state exceed the page size, write to HDFS @@ -262,7 +277,8 @@ private void writeOutTerminationState() throws HyracksDataException { } @Override - public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException { + public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb, IIndexCursor cursor) + throws HyracksDataException { try { if (vertex != null && vertex.hasUpdate()) { if (!dynamicStateLength) { @@ -271,12 +287,13 @@ public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb) th int offset = tupleRef.getFieldStart(1); bbos.setByteArray(data, offset); vertex.write(output); + ITreeIndexCursor tCursor = (ITreeIndexCursor) cursor; + tCursor.markCurrentTupleAsUpdated(); } else { // write the vertex id DataOutput tbOutput = cloneUpdateTb.getDataOutput(); vertex.getVertexId().write(tbOutput); cloneUpdateTb.addFieldEndOffset(); - // write the vertex value vertex.write(tbOutput); cloneUpdateTb.addFieldEndOffset(); diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/NoOpUpdateFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/NoOpUpdateFunctionFactory.java index 88577c295..8947c0134 100644 --- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/NoOpUpdateFunctionFactory.java +++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/NoOpUpdateFunctionFactory.java @@ -20,6 +20,7 @@ import edu.uci.ics.hyracks.api.exceptions.HyracksDataException; import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder; import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference; +import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor; import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunction; import edu.uci.ics.pregelix.dataflow.std.base.IUpdateFunctionFactory; @@ -57,7 +58,8 @@ public void close() throws HyracksDataException { } @Override - public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException { + public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb, IIndexCursor cursor) + throws HyracksDataException { } diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java index ca8ec01e4..9ddcce5b5 100644 --- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java +++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/function/StartComputeUpdateFunctionFactory.java @@ -31,6 +31,8 @@ import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder; import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender; import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference; +import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor; +import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor; import edu.uci.ics.pregelix.api.graph.GlobalAggregator; import edu.uci.ics.pregelix.api.graph.MsgList; import edu.uci.ics.pregelix.api.graph.Vertex; @@ -58,9 +60,9 @@ public IUpdateFunction createFunction() { private final ArrayTupleBuilder tbMsg = new ArrayTupleBuilder(2); private final ArrayTupleBuilder tbAlive = new ArrayTupleBuilder(2); private final ArrayTupleBuilder tbTerminate = new ArrayTupleBuilder(1); - private final ArrayTupleBuilder tbGlobalAggregate = new ArrayTupleBuilder(1); private final ArrayTupleBuilder tbInsert = new ArrayTupleBuilder(2); private final ArrayTupleBuilder tbDelete = new ArrayTupleBuilder(1); + private ArrayTupleBuilder tbGlobalAggregate; // for writing out to message channel private IFrameWriter writerMsg; @@ -77,7 +79,7 @@ public IUpdateFunction createFunction() { private IFrameWriter writerGlobalAggregate; private FrameTupleAppender appenderGlobalAggregate; private ByteBuffer bufferGlobalAggregate; - private GlobalAggregator aggregator; + private List aggregators; // for writing out the global aggregate private IFrameWriter writerTerminate; @@ -113,9 +115,14 @@ public IUpdateFunction createFunction() { public void open(IHyracksTaskContext ctx, RecordDescriptor rd, IFrameWriter... writers) throws HyracksDataException { this.conf = confFactory.createConfiguration(ctx); - this.dynamicStateLength = BspUtils.getDynamicVertexValueSize(conf); - this.aggregator = BspUtils.createGlobalAggregator(conf); - this.aggregator.init(); + //LSM index does not have in-place update + this.dynamicStateLength = BspUtils.getDynamicVertexValueSize(conf) || BspUtils.useLSM(conf);; + this.aggregators = BspUtils.createGlobalAggregators(conf); + for (int i = 0; i < aggregators.size(); i++) { + this.aggregators.get(i).init(); + } + + this.tbGlobalAggregate = new ArrayTupleBuilder(aggregators.size()); this.writerMsg = writers[0]; this.bufferMsg = ctx.allocateFrame(); @@ -201,7 +208,9 @@ public void process(Object[] tuple) throws HyracksDataException { /** * call the global aggregator */ - aggregator.step(vertex); + for (int i = 0; i < aggregators.size(); i++) { + aggregators.get(i).step(vertex); + } } @@ -223,13 +232,15 @@ public void close() throws HyracksDataException { private void writeOutGlobalAggregate() throws HyracksDataException { try { - /** - * get partial aggregate result and flush to the final - * aggregator - */ - Writable agg = aggregator.finishPartial(); - agg.write(tbGlobalAggregate.getDataOutput()); - tbGlobalAggregate.addFieldEndOffset(); + for (int i = 0; i < aggregators.size(); i++) { + /** + * get partial aggregate result and flush to the final + * aggregator + */ + Writable agg = aggregators.get(i).finishPartial(); + agg.write(tbGlobalAggregate.getDataOutput()); + tbGlobalAggregate.addFieldEndOffset(); + } if (!appenderGlobalAggregate.append(tbGlobalAggregate.getFieldEndOffsets(), tbGlobalAggregate.getByteArray(), 0, tbGlobalAggregate.getSize())) { // aggregate state exceed the page size, write to HDFS @@ -255,7 +266,8 @@ private void writeOutTerminationState() throws HyracksDataException { } @Override - public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb) throws HyracksDataException { + public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb, IIndexCursor cursor) + throws HyracksDataException { try { if (vertex != null && vertex.hasUpdate()) { if (!dynamicStateLength) { @@ -264,6 +276,8 @@ public void update(ITupleReference tupleRef, ArrayTupleBuilder cloneUpdateTb) th int offset = tupleRef.getFieldStart(1); bbos.setByteArray(data, offset); vertex.write(output); + ITreeIndexCursor tCursor = (ITreeIndexCursor) cursor; + tCursor.markCurrentTupleAsUpdated(); } else { // write the vertex id DataOutput tbOutput = cloneUpdateTb.getDataOutput(); diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/DatatypeHelper.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/DatatypeHelper.java index b121b5b82..e99fcb3ca 100644 --- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/DatatypeHelper.java +++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/DatatypeHelper.java @@ -17,32 +17,30 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import java.util.Map.Entry; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Writable; -import org.apache.hadoop.mapred.JobConf; import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer; import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor; import edu.uci.ics.hyracks.api.exceptions.HyracksDataException; +import edu.uci.ics.pregelix.api.util.ArrayListWritable; -@SuppressWarnings("deprecation") public class DatatypeHelper { private static final class WritableSerializerDeserializer implements ISerializerDeserializer { private static final long serialVersionUID = 1L; - private Class clazz; + private final Class clazz; + private transient Configuration conf; private T object; - private WritableSerializerDeserializer(Class clazz) { + private WritableSerializerDeserializer(Class clazz, Configuration conf) { this.clazz = clazz; + this.conf = conf; } - @SuppressWarnings("unchecked") + @SuppressWarnings({ "unchecked", "rawtypes" }) private T createInstance() throws HyracksDataException { // TODO remove "if", create a new WritableInstanceOperations class // that deals with Writables that don't have public constructors @@ -50,7 +48,11 @@ private T createInstance() throws HyracksDataException { return (T) NullWritable.get(); } try { - return clazz.newInstance(); + T t = clazz.newInstance(); + if (t instanceof ArrayListWritable) { + ((ArrayListWritable) t).setConf(conf); + } + return t; } catch (InstantiationException e) { throw new HyracksDataException(e); } catch (IllegalAccessException e) { @@ -85,42 +87,16 @@ public void serialize(T instance, DataOutput out) throws HyracksDataException { @SuppressWarnings({ "rawtypes", "unchecked" }) public static ISerializerDeserializer createSerializerDeserializer( - Class fClass) { - return new WritableSerializerDeserializer(fClass); + Class fClass, Configuration conf) { + return new WritableSerializerDeserializer(fClass, conf); } public static RecordDescriptor createKeyValueRecordDescriptor(Class keyClass, - Class valueClass) { + Class valueClass, Configuration conf) { @SuppressWarnings("rawtypes") ISerializerDeserializer[] fields = new ISerializerDeserializer[2]; - fields[0] = createSerializerDeserializer(keyClass); - fields[1] = createSerializerDeserializer(valueClass); + fields[0] = createSerializerDeserializer(keyClass, conf); + fields[1] = createSerializerDeserializer(valueClass, conf); return new RecordDescriptor(fields); } - - public static RecordDescriptor createOneFieldRecordDescriptor(Class fieldClass) { - @SuppressWarnings("rawtypes") - ISerializerDeserializer[] fields = new ISerializerDeserializer[1]; - fields[0] = createSerializerDeserializer(fieldClass); - return new RecordDescriptor(fields); - } - - public static JobConf map2JobConf(Map jobConfMap) { - JobConf jobConf; - synchronized (Configuration.class) { - jobConf = new JobConf(); - for (Entry entry : jobConfMap.entrySet()) { - jobConf.set(entry.getKey(), entry.getValue()); - } - } - return jobConf; - } - - public static Map jobConf2Map(JobConf jobConf) { - Map jobConfMap = new HashMap(); - for (Entry entry : jobConf) { - jobConfMap.put(entry.getKey(), entry.getValue()); - } - return jobConfMap; - } } \ No newline at end of file diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RuntimeHookFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RuntimeHookFactory.java index 5e8ac1eac..3151df246 100644 --- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RuntimeHookFactory.java +++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/RuntimeHookFactory.java @@ -14,6 +14,8 @@ */ package edu.uci.ics.pregelix.runtime.touchpoint; +import java.lang.reflect.Field; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TaskAttemptID; @@ -21,8 +23,6 @@ import edu.uci.ics.hyracks.api.context.IHyracksTaskContext; import edu.uci.ics.hyracks.api.exceptions.HyracksDataException; import edu.uci.ics.hyracks.hdfs.ContextFactory; -import edu.uci.ics.pregelix.api.graph.Vertex; -import edu.uci.ics.pregelix.api.util.BspUtils; import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory; import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHook; import edu.uci.ics.pregelix.dataflow.std.base.IRuntimeHookFactory; @@ -48,8 +48,12 @@ public void configure(IHyracksTaskContext ctx) throws HyracksDataException { try { TaskAttemptContext mapperContext = ctxFactory.createContext(conf, new TaskAttemptID()); mapperContext.getConfiguration().setClassLoader(ctx.getJobletContext().getClassLoader()); - Vertex.setContext(mapperContext); - BspUtils.setDefaultConfiguration(conf); + + ClassLoader cl = ctx.getJobletContext().getClassLoader(); + Class vClass = (Class) cl.loadClass("edu.uci.ics.pregelix.api.graph.Vertex"); + Field contextField = vClass.getDeclaredField("context"); + contextField.setAccessible(true); + contextField.set(null, mapperContext); } catch (Exception e) { throw new HyracksDataException(e); } diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexIdPartitionComputerFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexIdPartitionComputerFactory.java index 6fa6434c1..c9b67fb8f 100644 --- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexIdPartitionComputerFactory.java +++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/VertexIdPartitionComputerFactory.java @@ -16,6 +16,7 @@ import java.io.DataInputStream; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Writable; import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor; @@ -24,30 +25,39 @@ import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory; import edu.uci.ics.hyracks.api.exceptions.HyracksDataException; import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream; +import edu.uci.ics.pregelix.dataflow.base.IConfigurationFactory; import edu.uci.ics.pregelix.dataflow.std.base.ISerializerDeserializerFactory; public class VertexIdPartitionComputerFactory implements ITuplePartitionComputerFactory { private static final long serialVersionUID = 1L; private final ISerializerDeserializerFactory keyIOFactory; + private final IConfigurationFactory confFactory; - public VertexIdPartitionComputerFactory(ISerializerDeserializerFactory keyIOFactory) { + public VertexIdPartitionComputerFactory(ISerializerDeserializerFactory keyIOFactory, + IConfigurationFactory confFactory) { this.keyIOFactory = keyIOFactory; + this.confFactory = confFactory; } public ITuplePartitionComputer createPartitioner() { - return new ITuplePartitionComputer() { - private final ByteBufferInputStream bbis = new ByteBufferInputStream(); - private final DataInputStream dis = new DataInputStream(bbis); - private final ISerializerDeserializer keyIO = keyIOFactory.getSerializerDeserializer(); + try { + final Configuration conf = confFactory.createConfiguration(); + return new ITuplePartitionComputer() { + private final ByteBufferInputStream bbis = new ByteBufferInputStream(); + private final DataInputStream dis = new DataInputStream(bbis); + private final ISerializerDeserializer keyIO = keyIOFactory.getSerializerDeserializer(conf); - public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException { - int keyStart = accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength() - + accessor.getFieldStartOffset(tIndex, 0); - bbis.setByteBuffer(accessor.getBuffer(), keyStart); - K key = keyIO.deserialize(dis); - return Math.abs(key.hashCode() % nParts); - } - }; + public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException { + int keyStart = accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength() + + accessor.getFieldStartOffset(tIndex, 0); + bbis.setByteBuffer(accessor.getBuffer(), keyStart); + K key = keyIO.deserialize(dis); + return Math.abs(key.hashCode() % nParts); + } + }; + } catch (Exception e) { + throw new IllegalStateException(e); + } } } \ No newline at end of file diff --git a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/WritableSerializerDeserializerFactory.java b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/WritableSerializerDeserializerFactory.java index 435d0817f..c11ac5bf2 100644 --- a/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/WritableSerializerDeserializerFactory.java +++ b/pregelix/pregelix-runtime/src/main/java/edu/uci/ics/pregelix/runtime/touchpoint/WritableSerializerDeserializerFactory.java @@ -14,6 +14,7 @@ */ package edu.uci.ics.pregelix.runtime.touchpoint; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Writable; import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer; @@ -29,7 +30,7 @@ public WritableSerializerDeserializerFactory(Class clazz) { @SuppressWarnings({ "rawtypes", "unchecked" }) @Override - public ISerializerDeserializer getSerializerDeserializer() { - return DatatypeHelper.createSerializerDeserializer(clazz); + public ISerializerDeserializer getSerializerDeserializer(Configuration conf) { + return DatatypeHelper.createSerializerDeserializer(clazz, conf); } }