merged hyracks_asterix_stabilization r1634:1651
git-svn-id: https://hyracks.googlecode.com/svn/branches/hyracks_lsm_tree@1657 123451ca-8445-de46-9d55-352943316053
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
index 95b320f..a969372 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -42,5 +42,7 @@
WRITE_RESULT,
INSERT_DELETE,
INDEX_INSERT_DELETE,
- UPDATE
+ UPDATE,
+ INVERTED_INDEX_SEARCH,
+ PARTITIONINGSPLIT
}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
index 6def2b0..d86bf0f 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
@@ -42,7 +42,7 @@
*/
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(IDataSource<S> dataSource,
List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
- JobGenContext context, JobSpecification jobSpec) throws AlgebricksException;
+ IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec) throws AlgebricksException;
public boolean scannerOperatorIsLeaf(IDataSource<S> dataSource);
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/PartitioningSplitOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/PartitioningSplitOperator.java
index 37de0f9..6fe6bed 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/PartitioningSplitOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/PartitioningSplitOperator.java
@@ -15,6 +15,7 @@
package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical;
import java.util.ArrayList;
+import java.util.List;
import org.apache.commons.lang3.mutable.Mutable;
@@ -27,25 +28,48 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.typing.ITypingContext;
import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+import edu.uci.ics.hyracks.algebricks.runtime.operators.std.PartitioningSplitOperatorDescriptor;
+/**
+ * Partitions it's input based on a given list of expressions.
+ * Each expression is assumed to return true/false,
+ * and there is exactly one output branch per expression (optionally, plus one default branch).
+ * For each input tuple, the expressions are evaluated one-by-one,
+ * and the tuple is written to first output branch whose corresponding
+ * expression evaluates to true.
+ * If all expressions evaluate to false, then
+ * the tuple is written to the default output branch, if any was given.
+ * If no output branch was given, then such tuples are simply dropped.
+ * Given N expressions there may be N or N+1 output branches because the default output branch may be separate from the regular output branches.
+ */
public class PartitioningSplitOperator extends AbstractLogicalOperator {
- private Mutable<ILogicalExpression>[] expressions;
- private boolean hasDefault;
+ private final List<Mutable<ILogicalExpression>> expressions;
+ private final int defaultBranchIndex;
- public PartitioningSplitOperator(Mutable<ILogicalExpression>[] exprList, boolean defaultBranchDefined) {
- expressions = exprList;
- hasDefault = defaultBranchDefined;
+ public PartitioningSplitOperator(List<Mutable<ILogicalExpression>> expressions, int defaultBranchIndex) throws AlgebricksException {
+ this.expressions = expressions;
+ this.defaultBranchIndex = defaultBranchIndex;
+ // Check that the default output branch index is in [0, N], where N is the number of expressions.
+ if (defaultBranchIndex != PartitioningSplitOperatorDescriptor.NO_DEFAULT_BRANCH
+ && defaultBranchIndex > expressions.size()) {
+ throw new AlgebricksException("Default branch index out of bounds. Number of exprs given: "
+ + expressions.size() + ". The maximum default branch index may therefore be: " + expressions.size());
+ }
}
- public Mutable<ILogicalExpression>[] getExpressions() {
+ public List<Mutable<ILogicalExpression>> getExpressions() {
return expressions;
}
- public boolean hasDefault() {
- return hasDefault;
+ public int getDefaultBranchIndex() {
+ return defaultBranchIndex;
}
-
+
+ public int getNumOutputBranches() {
+ return (defaultBranchIndex == expressions.size()) ? expressions.size() + 1 : expressions.size();
+ }
+
@Override
public LogicalOperatorTag getOperatorTag() {
return LogicalOperatorTag.PARTITIONINGSPLIT;
@@ -65,8 +89,8 @@
@Override
public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
boolean b = false;
- for (int i = 0; i < expressions.length; i++) {
- if (visitor.transform(expressions[i])) {
+ for (int i = 0; i < expressions.size(); i++) {
+ if (visitor.transform(expressions.get(i))) {
b = true;
}
}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
index 0d20f22..b9a137f 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
@@ -34,13 +34,15 @@
public class UnnestMapOperator extends AbstractUnnestOperator {
private final List<Object> variableTypes; // TODO: get rid of this and
-
+ private final boolean propagateInput;
+
// deprecate UnnestMap
public UnnestMapOperator(List<LogicalVariable> variables, Mutable<ILogicalExpression> expression,
- List<Object> variableTypes) {
+ List<Object> variableTypes, boolean propagateInput) {
super(variables, expression);
this.variableTypes = variableTypes;
+ this.propagateInput = propagateInput;
}
@Override
@@ -61,10 +63,12 @@
@Override
public VariablePropagationPolicy getVariablePropagationPolicy() {
return new VariablePropagationPolicy() {
-
@Override
public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
throws AlgebricksException {
+ if (propagateInput) {
+ target.addAllVariables(sources[0]);
+ }
for (LogicalVariable v : variables) {
target.addVariable(v);
}
@@ -78,13 +82,28 @@
@Override
public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
- IVariableTypeEnvironment env = new NonPropagatingTypeEnvironment(ctx.getExpressionTypeComputer(),
- ctx.getMetadataProvider());
+ IVariableTypeEnvironment env = null;
+ if (propagateInput) {
+ env = createPropagatingAllInputsTypeEnvironment(ctx);
+ } else {
+ env = new NonPropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx.getMetadataProvider());
+ }
int n = variables.size();
for (int i = 0; i < n; i++) {
env.setVarType(variables.get(i), variableTypes.get(i));
}
return env;
}
+
+ public boolean propagatesInput() {
+ return propagateInput;
+ }
+
+ /*
+ @Override
+ public boolean isMap() {
+ return !propagateInput;
+ }
+ */
}
\ No newline at end of file
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index d845bf7..31061db 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -15,7 +15,6 @@
package edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -268,8 +267,7 @@
if (aop.getOperatorTag() != LogicalOperatorTag.PARTITIONINGSPLIT)
return Boolean.FALSE;
PartitioningSplitOperator partitionOpArg = (PartitioningSplitOperator) copyAndSubstituteVar(op, arg);
- boolean isomorphic = compareExpressions(Arrays.asList(op.getExpressions()),
- Arrays.asList(partitionOpArg.getExpressions()));
+ boolean isomorphic = compareExpressions(op.getExpressions(), partitionOpArg.getExpressions());
return isomorphic;
}
@@ -686,8 +684,8 @@
public ILogicalOperator visitPartitioningSplitOperator(PartitioningSplitOperator op, Void arg)
throws AlgebricksException {
ArrayList<Mutable<ILogicalExpression>> newExpressions = new ArrayList<Mutable<ILogicalExpression>>();
- deepCopyExpressionRefs(newExpressions, Arrays.asList(op.getExpressions()));
- return new PartitioningSplitOperator(newExpressions.toArray(new Mutable[0]), op.hasDefault());
+ deepCopyExpressionRefs(newExpressions, op.getExpressions());
+ return new PartitioningSplitOperator(newExpressions, op.getDefaultBranchIndex());
}
@Override
@@ -734,7 +732,7 @@
ArrayList<LogicalVariable> newInputList = new ArrayList<LogicalVariable>();
newInputList.addAll(op.getVariables());
return new UnnestMapOperator(newInputList, deepCopyExpressionRef(op.getExpressionRef()),
- new ArrayList<Object>(op.getVariableTypes()));
+ new ArrayList<Object>(op.getVariableTypes()), op.propagatesInput());
}
@Override
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java
index 0160449..1397a56 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AggregatePOperator.java
@@ -87,7 +87,7 @@
AggregateRuntimeFactory runtime = new AggregateRuntimeFactory(aggFactories);
// contribute one Asterix framewriter
- RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
builder.contributeMicroOperator(aggOp, runtime, recDesc);
// and contribute one edge from its child
ILogicalOperator src = aggOp.getInputs().get(0).getValue();
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AssignPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AssignPOperator.java
index 74402ec..f5bd9d9 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AssignPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/AssignPOperator.java
@@ -79,7 +79,7 @@
AssignRuntimeFactory runtime = new AssignRuntimeFactory(outColumns, evalFactories, projectionList);
// contribute one Asterix framewriter
- RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
builder.contributeMicroOperator(assign, runtime, recDesc);
// and contribute one edge from its child
ILogicalOperator src = assign.getInputs().get(0).getValue();
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
index 4617ff4..a2dac3f 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/DataSourceScanPOperator.java
@@ -24,6 +24,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
@@ -66,11 +67,11 @@
throws AlgebricksException {
DataSourceScanOperator scan = (DataSourceScanOperator) op;
IMetadataProvider mp = context.getMetadataProvider();
-
+ IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
List<LogicalVariable> vars = scan.getVariables();
List<LogicalVariable> projectVars = scan.getProjectVariables();
Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = mp.getScannerRuntime(dataSource, vars,
- projectVars, scan.isProjectPushed(), context, builder.getJobSpec());
+ projectVars, scan.isProjectPushed(), opSchema, typeEnv, context, builder.getJobSpec());
builder.contributeHyracksOperator(scan, p.first);
if (p.second != null) {
builder.contributeAlgebricksPartitionConstraint(p.first, p.second);
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java
index 34913da..55add37 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ExternalGroupByPOperator.java
@@ -201,7 +201,7 @@
IOperatorDescriptorRegistry spec = builder.getJobSpec();
IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(gbyCols,
aggOpInputEnv, context);
- RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
IBinaryHashFunctionFactory[] hashFunctionFactories = JobGenHelper.variablesToBinaryHashFunctionFactories(
gbyCols, aggOpInputEnv, context);
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
index 934d6c8..c737cc4 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/HybridHashJoinPOperator.java
@@ -97,7 +97,7 @@
Object t = env.getVarType(v);
comparatorFactories[i++] = bcfp.getBinaryComparatorFactory(t, true);
}
- RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
IOperatorDescriptorRegistry spec = builder.getJobSpec();
IOperatorDescriptor opDesc = null;
try {
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
index d6c6b85..563fcc5 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryHashJoinPOperator.java
@@ -87,7 +87,7 @@
Object t = env.getVarType(v);
comparatorFactories[i++] = bcfp.getBinaryComparatorFactory(t, true);
}
- RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
IOperatorDescriptorRegistry spec = builder.getJobSpec();
IOperatorDescriptor opDesc = null;
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryStableSortPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryStableSortPOperator.java
index 4ef5aad..ec2862e 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryStableSortPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InMemoryStableSortPOperator.java
@@ -52,7 +52,7 @@
public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
throws AlgebricksException {
- RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
int n = sortColumns.length;
int[] sortFields = new int[n];
IBinaryComparatorFactory[] comps = new IBinaryComparatorFactory[n];
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java
index 9546d47..e000b85 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java
@@ -93,7 +93,7 @@
IMetadataProvider mp = context.getMetadataProvider();
JobSpecification spec = builder.getJobSpec();
- RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(op.getInputs().get(0).getValue(), inputSchemas[0],
+ RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0],
context);
Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = null;
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
index edc85a9..477d257 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
@@ -71,7 +71,7 @@
InsertDeleteOperator insertDeleteOp = (InsertDeleteOperator) op;
IMetadataProvider mp = context.getMetadataProvider();
JobSpecification spec = builder.getJobSpec();
- RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(op.getInputs().get(0).getValue(), inputSchemas[0],
+ RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0],
context);
Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = null;
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java
index 9d6eccf..c159ffd 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/MicroPreclusteredGroupByPOperator.java
@@ -67,8 +67,8 @@
IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
columnList, env, context);
- RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
- RecordDescriptor inputRecordDesc = JobGenHelper.mkRecordDescriptor(op.getInputs().get(0).getValue(),
+ RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
+ RecordDescriptor inputRecordDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op.getInputs().get(0).getValue()),
inputSchemas[0], context);
MicroPreClusteredGroupRuntimeFactory runtime = new MicroPreClusteredGroupRuntimeFactory(keys,
comparatorFactories, aggregatorFactory, inputRecordDesc, recordDescriptor, null);
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NLJoinPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NLJoinPOperator.java
index aa8e560..bcd31a6 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NLJoinPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NLJoinPOperator.java
@@ -122,7 +122,7 @@
IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
throws AlgebricksException {
AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) op;
- RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
IOperatorSchema[] conditionInputSchemas = new IOperatorSchema[1];
conditionInputSchemas[0] = propagatedSchema;
IExpressionRuntimeProvider expressionRuntimeProvider = context.getExpressionRuntimeProvider();
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NestedTupleSourcePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NestedTupleSourcePOperator.java
index c069816..47a3dd2 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NestedTupleSourcePOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/NestedTupleSourcePOperator.java
@@ -67,7 +67,7 @@
throws AlgebricksException {
propagatedSchema.addAllVariables(outerPlanSchema);
NestedTupleSourceRuntimeFactory runtime = new NestedTupleSourceRuntimeFactory();
- RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
builder.contributeMicroOperator(op, runtime, recDesc);
}
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java
index dbf4287..29af97c 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreSortedDistinctByPOperator.java
@@ -111,7 +111,7 @@
IAggregatorDescriptorFactory aggregatorFactory = new SimpleAlgebricksAccumulatingAggregatorFactory(
aggFactories, keys, fdColumns);
- RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
PreclusteredGroupOperatorDescriptor opDesc = new PreclusteredGroupOperatorDescriptor(spec, keys,
comparatorFactories, aggregatorFactory, recordDescriptor);
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java
index 2d4537f..0849c7d 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/PreclusteredGroupByPOperator.java
@@ -82,7 +82,7 @@
IOperatorDescriptorRegistry spec = builder.getJobSpec();
IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
columnList, context.getTypeEnvironment(op), context);
- RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
PreclusteredGroupOperatorDescriptor opDesc = new PreclusteredGroupOperatorDescriptor(spec, keys,
comparatorFactories, aggregatorFactory, recordDescriptor);
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java
index f076c58..060e8d7 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/ReplicatePOperator.java
@@ -60,7 +60,7 @@
IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
throws AlgebricksException {
IOperatorDescriptorRegistry spec = builder.getJobSpec();
- RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
ReplicateOperator rop = (ReplicateOperator) op;
int outputArity = rop.getOutputArity();
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
index f84c134..b37c0ea 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/RunningAggregatePOperator.java
@@ -106,7 +106,7 @@
projectionList);
// contribute one Asterix framewriter
- RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
builder.contributeMicroOperator(ragg, runtime, recDesc);
// and contribute one edge from its child
ILogicalOperator src = ragg.getInputs().get(0).getValue();
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SinkPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SinkPOperator.java
index 92452f6..ccb61a2 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SinkPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SinkPOperator.java
@@ -57,7 +57,7 @@
public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
throws AlgebricksException {
- RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
SinkRuntimeFactory runtime = new SinkRuntimeFactory();
builder.contributeMicroOperator(op, runtime, recDesc);
ILogicalOperator src = op.getInputs().get(0).getValue();
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
index a556699..e89b8e0 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SinkWritePOperator.java
@@ -87,8 +87,8 @@
LogicalVariable v = varRef.getVariableReference();
columns[i++] = inputSchemas[0].findVariable(v);
}
- RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
- RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(op.getInputs().get(0).getValue(), inputSchemas[0],
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
+ RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0],
context);
IPrinterFactory[] pf = JobGenHelper.mkPrinterFactories(inputSchemas[0], context.getTypeEnvironment(op),
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StableSortPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StableSortPOperator.java
index 9ae44c7..df15f29 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StableSortPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StableSortPOperator.java
@@ -63,7 +63,7 @@
IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
throws AlgebricksException {
IOperatorDescriptorRegistry spec = builder.getJobSpec();
- RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
int n = sortColumns.length;
int[] sortFields = new int[n];
IBinaryComparatorFactory[] comps = new IBinaryComparatorFactory[n];
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamDiePOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamDiePOperator.java
index c0e4b69..73b9b79 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamDiePOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamDiePOperator.java
@@ -71,7 +71,7 @@
IVariableTypeEnvironment env = context.getTypeEnvironment(op);
IScalarEvaluatorFactory afterObjectsFact = expressionRuntimeProvider.createEvaluatorFactory(die
.getAfterObjects().getValue(), env, inputSchemas, context);
- RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
StreamDieRuntimeFactory runtime = new StreamDieRuntimeFactory(afterObjectsFact, null,
context.getBinaryIntegerInspectorFactory());
builder.contributeMicroOperator(die, runtime, recDesc);
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java
index 34127aa..11e24d7 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamLimitPOperator.java
@@ -83,7 +83,7 @@
ILogicalExpression offsetExpr = limit.getOffset().getValue();
IScalarEvaluatorFactory offsetFact = (offsetExpr == null) ? null : expressionRuntimeProvider
.createEvaluatorFactory(offsetExpr, env, inputSchemas, context);
- RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
StreamLimitRuntimeFactory runtime = new StreamLimitRuntimeFactory(maxObjectsFact, offsetFact, null,
context.getBinaryIntegerInspectorFactory());
builder.contributeMicroOperator(limit, runtime, recDesc);
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamProjectPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamProjectPOperator.java
index f454b8f..54c0505 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamProjectPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamProjectPOperator.java
@@ -62,7 +62,7 @@
projectionList[i++] = pos;
}
StreamProjectRuntimeFactory runtime = new StreamProjectRuntimeFactory(projectionList);
- RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
builder.contributeMicroOperator(project, runtime, recDesc);
ILogicalOperator src = project.getInputs().get(0).getValue();
builder.contributeGraphEdge(src, 0, project, 0);
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
index 891f72a..4352d1f 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StreamSelectPOperator.java
@@ -65,7 +65,7 @@
StreamSelectRuntimeFactory runtime = new StreamSelectRuntimeFactory(cond, null,
context.getBinaryBooleanInspectorFactory());
// contribute one Asterix framewriter
- RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
builder.contributeMicroOperator(select, runtime, recDesc);
// and contribute one edge from its child
ILogicalOperator src = select.getInputs().get(0).getValue();
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java
index b251c7d..52f8e0b 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/StringStreamingScriptPOperator.java
@@ -61,7 +61,7 @@
StringStreamingScriptDescription sssd = (StringStreamingScriptDescription) scriptDesc;
StringStreamingRuntimeFactory runtime = new StringStreamingRuntimeFactory(sssd.getCommand(),
sssd.getPrinterFactories(), sssd.getFieldDelimiter(), sssd.getParserFactory());
- RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, propagatedSchema, context);
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema, context);
builder.contributeMicroOperator(scriptOp, runtime, recDesc);
// and contribute one edge from its child
ILogicalOperator src = scriptOp.getInputs().get(0).getValue();
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java
index df84641..933acce 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/SubplanPOperator.java
@@ -89,7 +89,7 @@
AlgebricksPipeline[] subplans = compileSubplans(inputSchemas[0], subplan, opSchema, context);
assert (subplans.length == 1);
AlgebricksPipeline np = subplans[0];
- RecordDescriptor inputRecordDesc = JobGenHelper.mkRecordDescriptor(op.getInputs().get(0).getValue(),
+ RecordDescriptor inputRecordDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op.getInputs().get(0).getValue()),
inputSchemas[0], context);
INullWriterFactory[] nullWriterFactories = new INullWriterFactory[np.getOutputWidth()];
for (int i = 0; i < nullWriterFactories.length; i++) {
@@ -97,7 +97,7 @@
}
SubplanRuntimeFactory runtime = new SubplanRuntimeFactory(np, nullWriterFactories, inputRecordDesc, null);
- RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
builder.contributeMicroOperator(subplan, runtime, recDesc);
ILogicalOperator src = op.getInputs().get(0).getValue();
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
index 5322ed3..d3f7851 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
@@ -85,7 +85,7 @@
}
IOperatorDescriptorRegistry spec = builder.getJobSpec();
- RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
// at algebricks level, union all only accepts two inputs, although at
// hyracks
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnnestPOperator.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnnestPOperator.java
index 750b570..38ebec1 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnnestPOperator.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/UnnestPOperator.java
@@ -81,7 +81,7 @@
context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas, context);
int[] projectionList = JobGenHelper.projectAllVariables(opSchema);
UnnestRuntimeFactory unnestRuntime = new UnnestRuntimeFactory(outCol, unnestingFactory, projectionList);
- RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(op, opSchema, context);
+ RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema, context);
builder.contributeMicroOperator(unnest, unnestRuntime, recDesc);
ILogicalOperator src = unnest.getInputs().get(0).getValue();
builder.contributeGraphEdge(src, 0, unnest, 0);
diff --git a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
index 63e6e63..790fb93 100644
--- a/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
+++ b/hyracks-algebricks/hyracks-algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/jobgen/impl/JobGenHelper.java
@@ -15,10 +15,10 @@
package edu.uci.ics.hyracks.algebricks.core.jobgen.impl;
import java.util.Collection;
+import java.util.List;
import java.util.logging.Logger;
import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
@@ -40,25 +40,26 @@
private static final Logger LOGGER = Logger.getLogger(JobGenHelper.class.getName());
- @SuppressWarnings("unchecked")
- public static RecordDescriptor mkRecordDescriptor(ILogicalOperator op, IOperatorSchema opSchema,
- JobGenContext context) throws AlgebricksException {
- ISerializerDeserializer[] fields = new ISerializerDeserializer[opSchema.getSize()];
+ @SuppressWarnings("rawtypes")
+ public static RecordDescriptor mkRecordDescriptor(IVariableTypeEnvironment env, IOperatorSchema opSchema,
+ JobGenContext context) throws AlgebricksException {
+ ISerializerDeserializer[] fields = new ISerializerDeserializer[opSchema.getSize()];
+ ITypeTraits[] typeTraits = new ITypeTraits[opSchema.getSize()];
ISerializerDeserializerProvider sdp = context.getSerializerDeserializerProvider();
+ ITypeTraitProvider ttp = context.getTypeTraitProvider();
int i = 0;
- IVariableTypeEnvironment env = context.getTypeEnvironment(op);
for (LogicalVariable var : opSchema) {
Object t = env.getVarType(var);
if (t == null) {
LOGGER.warning("No type for variable " + var);
- // throw new AlgebricksException("No type for variable " + var);
}
fields[i] = sdp.getSerializerDeserializer(t);
+ typeTraits[i] = ttp.getTypeTrait(t);
i++;
}
- return new RecordDescriptor(fields);
+ return new RecordDescriptor(fields, typeTraits);
}
-
+
public static IPrinterFactory[] mkPrinterFactories(IOperatorSchema opSchema, IVariableTypeEnvironment env,
JobGenContext context, int[] printColumns) throws AlgebricksException {
IPrinterFactory[] pf = new IPrinterFactory[printColumns.length];
@@ -93,7 +94,7 @@
}
return funFactories;
}
-
+
public static IBinaryComparatorFactory[] variablesToAscBinaryComparatorFactories(
Collection<LogicalVariable> varLogical, IVariableTypeEnvironment env, JobGenContext context)
throws AlgebricksException {
@@ -106,6 +107,18 @@
}
return compFactories;
}
+
+ public static IBinaryComparatorFactory[] variablesToAscBinaryComparatorFactories(
+ List<LogicalVariable> varLogical, int start, int size, IVariableTypeEnvironment env, JobGenContext context)
+ throws AlgebricksException {
+ IBinaryComparatorFactory[] compFactories = new IBinaryComparatorFactory[size];
+ IBinaryComparatorFactoryProvider bcfProvider = context.getBinaryComparatorFactoryProvider();
+ for (int i = 0; i < size; i++) {
+ Object type = env.getVarType(varLogical.get(start + i));
+ compFactories[i] = bcfProvider.getBinaryComparatorFactory(type, true);
+ }
+ return compFactories;
+ }
public static INormalizedKeyComputerFactory variablesToAscNormalizedKeyComputerFactory(
Collection<LogicalVariable> varLogical, IVariableTypeEnvironment env, JobGenContext context)
@@ -131,6 +144,18 @@
}
return typeTraits;
}
+
+ public static ITypeTraits[] variablesToTypeTraits(
+ List<LogicalVariable> varLogical, int start, int size, IVariableTypeEnvironment env, JobGenContext context)
+ throws AlgebricksException {
+ ITypeTraits[] typeTraits = new ITypeTraits[size];
+ ITypeTraitProvider typeTraitProvider = context.getTypeTraitProvider();
+ for (int i = 0; i < size; i++) {
+ Object type = env.getVarType(varLogical.get(start + i));
+ typeTraits[i] = typeTraitProvider.getTypeTrait(type);
+ }
+ return typeTraits;
+ }
public static int[] projectAllVariables(IOperatorSchema opSchema) {
int[] projectionList = new int[opSchema.getSize()];
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
index bcc31fb..d105759 100644
--- a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
@@ -43,6 +43,7 @@
import edu.uci.ics.hyracks.algebricks.core.rewriter.base.AbstractRuleController;
import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
import edu.uci.ics.hyracks.algebricks.data.ISerializerDeserializerProvider;
+import edu.uci.ics.hyracks.algebricks.data.ITypeTraitProvider;
import edu.uci.ics.hyracks.algebricks.examples.piglet.ast.ASTNode;
import edu.uci.ics.hyracks.algebricks.examples.piglet.ast.AssignmentNode;
import edu.uci.ics.hyracks.algebricks.examples.piglet.ast.DumpNode;
@@ -65,6 +66,7 @@
import edu.uci.ics.hyracks.algebricks.examples.piglet.types.Schema;
import edu.uci.ics.hyracks.algebricks.examples.piglet.types.Type;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.job.JobSpecification;
public class PigletCompiler {
@@ -124,6 +126,11 @@
return null;
}
});
+ builder.setTypeTraitProvider(new ITypeTraitProvider() {
+ public ITypeTraits getTypeTrait(Object type) {
+ return null;
+ }
+ });
builder.setPrinterProvider(PigletPrinterFactoryProvider.INSTANCE);
builder.setExpressionRuntimeProvider(new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter(
new PigletExpressionJobGen()));
diff --git a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
index 7a4d71c4..d678803 100644
--- a/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
+++ b/hyracks-algebricks/hyracks-algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
@@ -62,9 +62,13 @@
@SuppressWarnings("unchecked")
@Override
- public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(IDataSource<String> dataSource,
- List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
- JobGenContext context, JobSpecification jobSpec) throws AlgebricksException {
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(
+ IDataSource<String> dataSource,
+ List<LogicalVariable> scanVariables,
+ List<LogicalVariable> projectVariables, boolean projectPushed,
+ IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
+ JobGenContext context, JobSpecification jobSpec)
+ throws AlgebricksException {
PigletFileDataSource ds = (PigletFileDataSource) dataSource;
FileSplit[] fileSplits = ds.getFileSplits();
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateAssignsRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateAssignsRule.java
index 042c0aa..e0220cc 100644
--- a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateAssignsRule.java
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateAssignsRule.java
@@ -49,7 +49,6 @@
if (op2.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
return false;
}
-
AssignOperator assign2 = (AssignOperator) op2;
HashSet<LogicalVariable> used1 = new HashSet<LogicalVariable>();
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
index 33ecd97..ff9d921 100644
--- a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
@@ -14,8 +14,6 @@
*/
package edu.uci.ics.hyracks.algebricks.rewriter.rules;
-import java.util.List;
-
import org.apache.commons.lang3.mutable.Mutable;
import org.apache.commons.lang3.mutable.MutableObject;
@@ -32,6 +30,14 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+/**
+ * Matches the following operator pattern:
+ * (select) <-- ((assign)* <-- (select)*)+
+ *
+ * Consolidates the selects to:
+ * (select) <-- (assign)*
+ *
+ */
public class ConsolidateSelectsRule implements IAlgebraicRewriteRule {
@Override
@@ -41,39 +47,63 @@
@Override
public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
- AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
return false;
}
- SelectOperator select = (SelectOperator) op;
-
- AbstractLogicalOperator op2 = (AbstractLogicalOperator) select.getInputs().get(0).getValue();
- if (op2.getOperatorTag() != LogicalOperatorTag.SELECT) {
- return false;
- }
+ SelectOperator firstSelect = (SelectOperator) op;
IFunctionInfo andFn = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND);
- AbstractFunctionCallExpression conj = new ScalarFunctionCallExpression(andFn);
- conj.getArguments().add(new MutableObject<ILogicalExpression>(select.getCondition().getValue()));
- conj.getArguments().add(((SelectOperator) op2).getCondition());
-
- Mutable<ILogicalOperator> botOpRef = select.getInputs().get(0);
- boolean more = true;
- while (more) {
- botOpRef = botOpRef.getValue().getInputs().get(0);
- AbstractLogicalOperator botOp = (AbstractLogicalOperator) botOpRef.getValue();
- if (botOp.getOperatorTag() == LogicalOperatorTag.SELECT) {
- conj.getArguments().add(((SelectOperator) botOp).getCondition());
- } else {
- more = false;
+ // New conjuncts for consolidated select.
+ AbstractFunctionCallExpression conj = null;
+ AbstractLogicalOperator topMostOp = null;
+ AbstractLogicalOperator selectParent = null;
+ AbstractLogicalOperator nextSelect = firstSelect;
+ do {
+ // Skip through assigns.
+ do {
+ selectParent = nextSelect;
+ nextSelect = (AbstractLogicalOperator) selectParent.getInputs().get(0).getValue();
+ } while (nextSelect.getOperatorTag() == LogicalOperatorTag.ASSIGN);
+ // Stop if the child op is not a select.
+ if (nextSelect.getOperatorTag() != LogicalOperatorTag.SELECT) {
+ break;
+ }
+ // Remember the top-most op that we are not removing.
+ topMostOp = selectParent;
+
+ // Initialize the new conjuncts, if necessary.
+ if (conj == null) {
+ conj = new ScalarFunctionCallExpression(andFn);
+ // Add the first select's condition.
+ conj.getArguments().add(new MutableObject<ILogicalExpression>(firstSelect.getCondition().getValue()));
}
+
+ // Consolidate all following selects.
+ do {
+ // Add the condition nextSelect to the new list of conjuncts.
+ conj.getArguments().add(((SelectOperator) nextSelect).getCondition());
+ selectParent = nextSelect;
+ nextSelect = (AbstractLogicalOperator) nextSelect.getInputs().get(0).getValue();
+ } while (nextSelect.getOperatorTag() == LogicalOperatorTag.SELECT);
+
+ // Hook up the input of the top-most remaining op if necessary.
+ if (topMostOp.getOperatorTag() == LogicalOperatorTag.ASSIGN || topMostOp == firstSelect) {
+ topMostOp.getInputs().set(0, selectParent.getInputs().get(0));
+ }
+
+ // Prepare for next iteration.
+ nextSelect = selectParent;
+ } while (true);
+
+ // Did we consolidate any selects?
+ if (conj == null) {
+ return false;
}
- select.getCondition().setValue(conj);
- List<Mutable<ILogicalOperator>> selInptList = select.getInputs();
- selInptList.clear();
- selInptList.add(botOpRef);
- context.computeAndSetTypeEnvironmentForOperator(select);
+
+ // Set the new conjuncts.
+ firstSelect.getCondition().setValue(conj);
+ context.computeAndSetTypeEnvironmentForOperator(firstSelect);
return true;
}
-
}
diff --git a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushProjectDownRule.java b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushProjectDownRule.java
index 26d294b..f223844 100644
--- a/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushProjectDownRule.java
+++ b/hyracks-algebricks/hyracks-algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/PushProjectDownRule.java
@@ -82,7 +82,8 @@
if (op2.getOperatorTag() == LogicalOperatorTag.EMPTYTUPLESOURCE
|| op2.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE
|| op2.getOperatorTag() == LogicalOperatorTag.PROJECT
- || op2.getOperatorTag() == LogicalOperatorTag.REPLICATE) {
+ || op2.getOperatorTag() == LogicalOperatorTag.REPLICATE
+ || op2.getOperatorTag() == LogicalOperatorTag.UNIONALL) {
return new Pair<Boolean, Boolean>(false, false);
}
if (!op2.isMap()) {
diff --git a/hyracks-algebricks/hyracks-algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java b/hyracks-algebricks/hyracks-algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java
index e978ade..5aaf196 100644
--- a/hyracks-algebricks/hyracks-algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java
+++ b/hyracks-algebricks/hyracks-algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/meta/AlgebricksMetaOperatorDescriptor.java
@@ -119,7 +119,7 @@
RecordDescriptor pipelineOutputRecordDescriptor = outputArity > 0 ? AlgebricksMetaOperatorDescriptor.this.recordDescriptors[0]
: null;
RecordDescriptor pipelineInputRecordDescriptor = recordDescProvider.getInputRecordDescriptor(
- AlgebricksMetaOperatorDescriptor.this.getOperatorId(), 0);
+ AlgebricksMetaOperatorDescriptor.this.getActivityId(), 0);
PipelineAssembler pa = new PipelineAssembler(pipeline, inputArity, outputArity,
pipelineInputRecordDescriptor, pipelineOutputRecordDescriptor);
try {
diff --git a/hyracks-algebricks/hyracks-algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/PartitioningSplitOperatorDescriptor.java b/hyracks-algebricks/hyracks-algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/PartitioningSplitOperatorDescriptor.java
new file mode 100644
index 0000000..fa05ea5
--- /dev/null
+++ b/hyracks-algebricks/hyracks-algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/operators/std/PartitioningSplitOperatorDescriptor.java
@@ -0,0 +1,160 @@
+package edu.uci.ics.hyracks.algebricks.runtime.operators.std;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspector;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputOperatorNodePushable;
+
+public class PartitioningSplitOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+ private static final long serialVersionUID = 1L;
+ public static int NO_DEFAULT_BRANCH = -1;
+
+ private final ICopyEvaluatorFactory[] evalFactories;
+ private final IBinaryBooleanInspector boolInspector;
+ private final int defaultBranchIndex;
+
+ public PartitioningSplitOperatorDescriptor(IOperatorDescriptorRegistry spec, ICopyEvaluatorFactory[] evalFactories,
+ IBinaryBooleanInspector boolInspector, int defaultBranchIndex, RecordDescriptor rDesc) {
+ super(spec, 1, (defaultBranchIndex == evalFactories.length) ? evalFactories.length + 1 : evalFactories.length);
+ for (int i = 0; i < evalFactories.length; i++) {
+ recordDescriptors[i] = rDesc;
+ }
+ this.evalFactories = evalFactories;
+ this.boolInspector = boolInspector;
+ this.defaultBranchIndex = defaultBranchIndex;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+ throws HyracksDataException {
+ return new AbstractUnaryInputOperatorNodePushable() {
+ private final IFrameWriter[] writers = new IFrameWriter[outputArity];
+ private final ByteBuffer[] writeBuffers = new ByteBuffer[outputArity];
+ private final ICopyEvaluator[] evals = new ICopyEvaluator[outputArity];
+ private final ArrayBackedValueStorage evalBuf = new ArrayBackedValueStorage();
+ private final RecordDescriptor inOutRecDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+ private final FrameTupleAccessor accessor = new FrameTupleAccessor(ctx.getFrameSize(), inOutRecDesc);
+ private final FrameTupleReference frameTuple = new FrameTupleReference();
+
+ private final FrameTupleAppender tupleAppender = new FrameTupleAppender(ctx.getFrameSize());
+ private final ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(inOutRecDesc.getFieldCount());
+ private final DataOutput tupleDos = tupleBuilder.getDataOutput();
+
+ @Override
+ public void close() throws HyracksDataException {
+ // Flush (possibly not full) buffers that have data, and close writers.
+ for (int i = 0; i < outputArity; i++) {
+ tupleAppender.reset(writeBuffers[i], false);
+ if (tupleAppender.getTupleCount() > 0) {
+ FrameUtils.flushFrame(writeBuffers[i], writers[i]);
+ }
+ writers[i].close();
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ for (IFrameWriter writer : writers) {
+ writer.fail();
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ accessor.reset(buffer);
+ int tupleCount = accessor.getTupleCount();
+ for (int i = 0; i < tupleCount; i++) {
+ frameTuple.reset(accessor, i);
+ boolean found = false;
+ for (int j = 0; j < evals.length; j++) {
+ try {
+ evalBuf.reset();
+ evals[j].evaluate(frameTuple);
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ found = boolInspector.getBooleanValue(evalBuf.getByteArray(), 0, 1);
+ if (found) {
+ copyAndAppendTuple(j);
+ break;
+ }
+ }
+ // Optionally write to default output branch.
+ if (!found && defaultBranchIndex != NO_DEFAULT_BRANCH) {
+ copyAndAppendTuple(defaultBranchIndex);
+ }
+ }
+ }
+
+ private void copyAndAppendTuple(int outputIndex) throws HyracksDataException {
+ // Copy tuple into tuple builder.
+ try {
+ tupleBuilder.reset();
+ for (int i = 0; i < frameTuple.getFieldCount(); i++) {
+ tupleDos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i),
+ frameTuple.getFieldLength(i));
+ tupleBuilder.addFieldEndOffset();
+ }
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ // Append to frame.
+ tupleAppender.reset(writeBuffers[outputIndex], false);
+ if (!tupleAppender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
+ FrameUtils.flushFrame(writeBuffers[outputIndex], writers[outputIndex]);
+ tupleAppender.reset(writeBuffers[outputIndex], true);
+ if (!tupleAppender.append(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0, tupleBuilder.getSize())) {
+ throw new IllegalStateException();
+ }
+ }
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ for (IFrameWriter writer : writers) {
+ writer.open();
+ }
+ // Create write buffers.
+ for (int i = 0; i < outputArity; i++) {
+ writeBuffers[i] = ctx.allocateFrame();
+ // Make sure to clear all buffers, since we are reusing the tupleAppender.
+ tupleAppender.reset(writeBuffers[i], true);
+ }
+ // Create evaluators for partitioning.
+ try {
+ for (int i = 0; i < evalFactories.length; i++) {
+ evals[i] = evalFactories[i].createEvaluator(evalBuf);
+ }
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+ writers[index] = writer;
+ }
+ };
+ }
+}
+
diff --git a/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java b/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java
index 922b0fd..b5c4b47 100644
--- a/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java
+++ b/hyracks-algebricks/hyracks-algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java
@@ -79,9 +79,8 @@
}
public static void runJob(JobSpecification spec) throws Exception {
- JobId jobId = hcc.createJob(AlgebricksConfig.HYRACKS_APP_NAME, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
AlgebricksConfig.ALGEBRICKS_LOGGER.info(spec.toJSON().toString());
- hcc.start(jobId);
+ JobId jobId = hcc.startJob(AlgebricksConfig.HYRACKS_APP_NAME, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
AlgebricksConfig.ALGEBRICKS_LOGGER.info(jobId.toString());
hcc.waitForCompletion(jobId);
}
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java
index 4250112..2792d29 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/application/ICCApplicationContext.java
@@ -18,21 +18,16 @@
import edu.uci.ics.hyracks.api.context.ICCContext;
import edu.uci.ics.hyracks.api.job.IJobLifecycleListener;
-import edu.uci.ics.hyracks.api.job.IJobSpecificationFactory;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
/**
* Application Context at the Cluster Controller for an application.
*
* @author vinayakb
- *
*/
public interface ICCApplicationContext extends IApplicationContext {
/**
* Sets the state that must be distributed by the infrastructure to all the
- * NC application contects. Any state set by calling thsi method in the
- * {@link ICCBootstrap#start()} call is made available to all the
- * {@link INCApplicationContext} objects at each Node Controller. The state
+ * NC application contects. Any state set by calling thsi method in the {@link ICCBootstrap#start()} call is made available to all the {@link INCApplicationContext} objects at each Node Controller. The state
* is then available to be inspected by the application at the NC during or
* after the {@link INCBootstrap#start()} call.
*
@@ -42,17 +37,6 @@
public void setDistributedState(Serializable state);
/**
- * A factory class specific to this application that may accept incoming
- * {@link JobSpecification} and produce modified {@link JobSpecification}
- * that is executed on the cluster. If a {@link IJobSpecificationFactory} is
- * not set, the incoming {@link JobSpecification} is executed unmodified.
- *
- * @param jobSpecFactory
- * - The Job Specification Factory.
- */
- public void setJobSpecificationFactory(IJobSpecificationFactory jobSpecFactory);
-
- /**
* A listener that listens to Job Lifecycle events at the Cluster
* Controller.
*
@@ -66,6 +50,4 @@
* @return The Cluster Controller Context.
*/
public ICCContext getCCContext();
-
-
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
index a7f324d..db74c41 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
@@ -105,37 +105,6 @@
}
}
- public static class CreateJobFunction extends Function {
- private static final long serialVersionUID = 1L;
-
- private final String appName;
- private final byte[] jobSpec;
- private final EnumSet<JobFlag> jobFlags;
-
- public CreateJobFunction(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) {
- this.appName = appName;
- this.jobSpec = jobSpec;
- this.jobFlags = jobFlags;
- }
-
- @Override
- public FunctionId getFunctionId() {
- return FunctionId.CREATE_JOB;
- }
-
- public String getAppName() {
- return appName;
- }
-
- public byte[] getJobSpec() {
- return jobSpec;
- }
-
- public EnumSet<JobFlag> getJobFlags() {
- return jobFlags;
- }
- }
-
public static class GetJobStatusFunction extends Function {
private static final long serialVersionUID = 1L;
@@ -158,10 +127,14 @@
public static class StartJobFunction extends Function {
private static final long serialVersionUID = 1L;
- private final JobId jobId;
+ private final String appName;
+ private final byte[] jobSpec;
+ private final EnumSet<JobFlag> jobFlags;
- public StartJobFunction(JobId jobId) {
- this.jobId = jobId;
+ public StartJobFunction(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) {
+ this.appName = appName;
+ this.jobSpec = jobSpec;
+ this.jobFlags = jobFlags;
}
@Override
@@ -169,8 +142,16 @@
return FunctionId.START_JOB;
}
- public JobId getJobId() {
- return jobId;
+ public String getAppName() {
+ return appName;
+ }
+
+ public byte[] getJobSpec() {
+ return jobSpec;
+ }
+
+ public EnumSet<JobFlag> getJobFlags() {
+ return jobFlags;
}
}
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
index 602288c..7413951 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
@@ -61,13 +61,6 @@
}
@Override
- public JobId createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
- HyracksClientInterfaceFunctions.CreateJobFunction cjf = new HyracksClientInterfaceFunctions.CreateJobFunction(
- appName, jobSpec, jobFlags);
- return (JobId) rpci.call(ipcHandle, cjf);
- }
-
- @Override
public JobStatus getJobStatus(JobId jobId) throws Exception {
HyracksClientInterfaceFunctions.GetJobStatusFunction gjsf = new HyracksClientInterfaceFunctions.GetJobStatusFunction(
jobId);
@@ -75,10 +68,10 @@
}
@Override
- public void startJob(JobId jobId) throws Exception {
+ public JobId startJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
HyracksClientInterfaceFunctions.StartJobFunction sjf = new HyracksClientInterfaceFunctions.StartJobFunction(
- jobId);
- rpci.call(ipcHandle, sjf);
+ appName, jobSpec, jobFlags);
+ return (JobId) rpci.call(ipcHandle, sjf);
}
@Override
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
index 6ff3f7e..33d76ec 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
@@ -94,23 +94,18 @@
}
@Override
- public JobId createJob(String appName, JobSpecification jobSpec) throws Exception {
- return createJob(appName, jobSpec, EnumSet.noneOf(JobFlag.class));
- }
-
- @Override
- public JobId createJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
- return hci.createJob(appName, JavaSerializationUtils.serialize(jobSpec), jobFlags);
- }
-
- @Override
public JobStatus getJobStatus(JobId jobId) throws Exception {
return hci.getJobStatus(jobId);
}
@Override
- public void start(JobId jobId) throws Exception {
- hci.startJob(jobId);
+ public JobId startJob(String appName, JobSpecification jobSpec) throws Exception {
+ return hci.startJob(appName, JavaSerializationUtils.serialize(jobSpec), EnumSet.noneOf(JobFlag.class));
+ }
+
+ @Override
+ public JobId startJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception {
+ return hci.startJob(appName, JavaSerializationUtils.serialize(jobSpec), jobFlags);
}
@Override
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
index 37a746b..72a87c2 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
@@ -27,7 +27,6 @@
* Interface used by clients to communicate with the Hyracks Cluster Controller.
*
* @author vinayakb
- *
*/
public interface IHyracksClientConnection {
/**
@@ -51,35 +50,6 @@
public void destroyApplication(String appName) throws Exception;
/**
- * Creates a Job Instance in the specified Hyracks application using the
- * specified {@link JobSpecification}.
- *
- * @param appName
- * Name of the application
- * @param jobSpec
- * Job Specification
- * @return
- * @throws Exception
- */
- public JobId createJob(String appName, JobSpecification jobSpec) throws Exception;
-
- /**
- * Creates a Job Instance in the specified Hyracks application using the
- * specified {@link JobSpecification}. The specified flags are used to
- * configure the Job creation process.
- *
- * @param appName
- * Name of the application
- * @param jobSpec
- * Job Specification
- * @param jobFlags
- * Flags
- * @return
- * @throws Exception
- */
- public JobId createJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
-
- /**
* Gets the status of the specified Job.
*
* @param jobId
@@ -92,11 +62,26 @@
/**
* Start the specified Job.
*
- * @param jobId
- * JobId of the Job.
+ * @param appName
+ * Name of the application
+ * @param jobSpec
+ * Job Specification
* @throws Exception
*/
- public void start(JobId jobId) throws Exception;
+ public JobId startJob(String appName, JobSpecification jobSpec) throws Exception;
+
+ /**
+ * Start the specified Job.
+ *
+ * @param appName
+ * Name of the application
+ * @param jobSpec
+ * Job Specification
+ * @param jobFlags
+ * Flags
+ * @throws Exception
+ */
+ public JobId startJob(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
/**
* Waits until the specified job has completed, either successfully or has
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
index 866d307..43dee05 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
@@ -30,11 +30,9 @@
public void destroyApplication(String appName) throws Exception;
- public JobId createJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
-
public JobStatus getJobStatus(JobId jobId) throws Exception;
- public void startJob(JobId jobId) throws Exception;
+ public JobId startJob(String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags) throws Exception;
public void waitForCompletion(JobId jobId) throws Exception;
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityGraphBuilder.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityGraphBuilder.java
index 56870b2..bbda23c 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityGraphBuilder.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/IActivityGraphBuilder.java
@@ -15,7 +15,7 @@
package edu.uci.ics.hyracks.api.dataflow;
public interface IActivityGraphBuilder {
- public void addActivity(IActivity task);
+ public void addActivity(IOperatorDescriptor op, IActivity task);
public void addBlockingEdge(IActivity blocker, IActivity blocked);
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IRecordDescriptorProvider.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IRecordDescriptorProvider.java
index 96d43a7..c480446 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IRecordDescriptorProvider.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IRecordDescriptorProvider.java
@@ -14,10 +14,10 @@
*/
package edu.uci.ics.hyracks.api.dataflow.value;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
public interface IRecordDescriptorProvider {
- public RecordDescriptor getInputRecordDescriptor(OperatorDescriptorId opId, int inputIndex);
+ public RecordDescriptor getInputRecordDescriptor(ActivityId aid, int inputIndex);
- public RecordDescriptor getOutputRecordDescriptor(OperatorDescriptorId opId, int outputIndex);
+ public RecordDescriptor getOutputRecordDescriptor(ActivityId aid, int outputIndex);
}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSpecificationFactory.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSpecificationFactory.java
deleted file mode 100644
index 722aaae..0000000
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/IJobSpecificationFactory.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Copyright 2009-2010 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.api.job;
-
-import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
-import edu.uci.ics.hyracks.api.application.ICCBootstrap;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-
-public interface IJobSpecificationFactory {
- public JobSpecification createJobSpecification(byte[] bytes, ICCBootstrap bootstrap, ICCApplicationContext appCtx)
- throws HyracksException;
-}
\ No newline at end of file
diff --git a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java
index 2b6d361..3c6a547 100644
--- a/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java
+++ b/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobActivityGraph.java
@@ -15,7 +15,6 @@
package edu.uci.ics.hyracks.api.job;
import java.io.Serializable;
-import java.util.ArrayList;
import java.util.EnumSet;
import java.util.HashMap;
import java.util.List;
@@ -31,8 +30,7 @@
import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
import edu.uci.ics.hyracks.api.dataflow.IActivity;
import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
+import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
public class JobActivityGraph implements Serializable {
@@ -40,54 +38,63 @@
private final String appName;
- private final JobSpecification jobSpec;
-
private final EnumSet<JobFlag> jobFlags;
- private final Map<ActivityId, IActivity> activityNodes;
+ private final Map<ActivityId, IActivity> activityMap;
+
+ private final Map<ConnectorDescriptorId, IConnectorDescriptor> connectorMap;
+
+ private final Map<ConnectorDescriptorId, RecordDescriptor> connectorRecordDescriptorMap;
+
+ private final Map<ActivityId, List<IConnectorDescriptor>> activityInputMap;
+
+ private final Map<ActivityId, List<IConnectorDescriptor>> activityOutputMap;
+
+ private final Map<ConnectorDescriptorId, Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>>> connectorActivityMap;
private final Map<ActivityId, Set<ActivityId>> blocker2blockedMap;
private final Map<ActivityId, Set<ActivityId>> blocked2blockerMap;
- private final Map<OperatorDescriptorId, Set<ActivityId>> operatorActivityMap;
+ private IConnectorPolicyAssignmentPolicy connectorPolicyAssignmentPolicy;
- private final Map<ActivityId, List<Integer>> activityInputMap;
+ private int maxReattempts;
- private final Map<ActivityId, List<Integer>> activityOutputMap;
+ private IJobletEventListenerFactory jobletEventListenerFactory;
- private final Map<OperatorDescriptorId, List<ActivityId>> operatorInputMap;
+ private IGlobalJobDataFactory globalJobDataFactory;
- private final Map<OperatorDescriptorId, List<ActivityId>> operatorOutputMap;
-
- public JobActivityGraph(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) {
+ public JobActivityGraph(String appName, EnumSet<JobFlag> jobFlags) {
this.appName = appName;
- this.jobSpec = jobSpec;
this.jobFlags = jobFlags;
- activityNodes = new HashMap<ActivityId, IActivity>();
+ activityMap = new HashMap<ActivityId, IActivity>();
+ connectorMap = new HashMap<ConnectorDescriptorId, IConnectorDescriptor>();
+ connectorRecordDescriptorMap = new HashMap<ConnectorDescriptorId, RecordDescriptor>();
+ activityInputMap = new HashMap<ActivityId, List<IConnectorDescriptor>>();
+ activityOutputMap = new HashMap<ActivityId, List<IConnectorDescriptor>>();
+ connectorActivityMap = new HashMap<ConnectorDescriptorId, Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>>>();
blocker2blockedMap = new HashMap<ActivityId, Set<ActivityId>>();
blocked2blockerMap = new HashMap<ActivityId, Set<ActivityId>>();
- operatorActivityMap = new HashMap<OperatorDescriptorId, Set<ActivityId>>();
- activityInputMap = new HashMap<ActivityId, List<Integer>>();
- activityOutputMap = new HashMap<ActivityId, List<Integer>>();
- operatorInputMap = new HashMap<OperatorDescriptorId, List<ActivityId>>();
- operatorOutputMap = new HashMap<OperatorDescriptorId, List<ActivityId>>();
}
public String getApplicationName() {
return appName;
}
- public JobSpecification getJobSpecification() {
- return jobSpec;
- }
-
public EnumSet<JobFlag> getJobFlags() {
return jobFlags;
}
- public Map<ActivityId, IActivity> getActivityNodeMap() {
- return activityNodes;
+ public Map<ActivityId, IActivity> getActivityMap() {
+ return activityMap;
+ }
+
+ public Map<ConnectorDescriptorId, IConnectorDescriptor> getConnectorMap() {
+ return connectorMap;
+ }
+
+ public Map<ConnectorDescriptorId, RecordDescriptor> getConnectorRecordDescriptorMap() {
+ return connectorRecordDescriptorMap;
}
public Map<ActivityId, Set<ActivityId>> getBlocker2BlockedMap() {
@@ -98,106 +105,64 @@
return blocked2blockerMap;
}
- public Map<OperatorDescriptorId, Set<ActivityId>> getOperatorActivityMap() {
- return operatorActivityMap;
- }
-
- public Map<ActivityId, List<Integer>> getActivityInputMap() {
+ public Map<ActivityId, List<IConnectorDescriptor>> getActivityInputMap() {
return activityInputMap;
}
- public Map<ActivityId, List<Integer>> getActivityOutputMap() {
+ public Map<ActivityId, List<IConnectorDescriptor>> getActivityOutputMap() {
return activityOutputMap;
}
- public Map<OperatorDescriptorId, List<ActivityId>> getOperatorInputMap() {
- return operatorInputMap;
- }
-
- public Map<OperatorDescriptorId, List<ActivityId>> getOperatorOutputMap() {
- return operatorOutputMap;
- }
-
- public List<IConnectorDescriptor> getActivityInputConnectorDescriptors(ActivityId hanId) {
- List<Integer> inputIndexes = activityInputMap.get(hanId);
- if (inputIndexes == null) {
- return null;
- }
- OperatorDescriptorId ownerId = hanId.getOperatorDescriptorId();
- List<IConnectorDescriptor> inputs = new ArrayList<IConnectorDescriptor>();
- for (Integer i : inputIndexes) {
- inputs.add(jobSpec.getInputConnectorDescriptor(ownerId, i));
- }
- return inputs;
- }
-
- public List<IConnectorDescriptor> getActivityOutputConnectorDescriptors(ActivityId hanId) {
- List<Integer> outputIndexes = activityOutputMap.get(hanId);
- if (outputIndexes == null) {
- return null;
- }
- OperatorDescriptorId ownerId = hanId.getOperatorDescriptorId();
- List<IConnectorDescriptor> outputs = new ArrayList<IConnectorDescriptor>();
- for (Integer i : outputIndexes) {
- outputs.add(jobSpec.getOutputConnectorDescriptor(ownerId, i));
- }
- return outputs;
+ public Map<ConnectorDescriptorId, Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>>> getConnectorActivityMap() {
+ return connectorActivityMap;
}
public ActivityId getConsumerActivity(ConnectorDescriptorId cdId) {
- Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connEdge = jobSpec
- .getConnectorOperatorMap().get(cdId);
-
- OperatorDescriptorId consumerOpId = connEdge.getRight().getLeft().getOperatorId();
- int consumerInputIdx = connEdge.getRight().getRight();
-
- for (ActivityId anId : operatorActivityMap.get(consumerOpId)) {
- List<Integer> anInputs = activityInputMap.get(anId);
- if (anInputs != null) {
- for (Integer idx : anInputs) {
- if (idx.intValue() == consumerInputIdx) {
- return anId;
- }
- }
- }
- }
- return null;
+ Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>> connEdge = connectorActivityMap.get(cdId);
+ return connEdge.getRight().getLeft().getActivityId();
}
public ActivityId getProducerActivity(ConnectorDescriptorId cdId) {
- Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> connEdge = jobSpec
- .getConnectorOperatorMap().get(cdId);
-
- OperatorDescriptorId producerOpId = connEdge.getLeft().getLeft().getOperatorId();
- int producerInputIdx = connEdge.getLeft().getRight();
-
- for (ActivityId anId : operatorActivityMap.get(producerOpId)) {
- List<Integer> anOutputs = activityOutputMap.get(anId);
- if (anOutputs != null) {
- for (Integer idx : anOutputs) {
- if (idx.intValue() == producerInputIdx) {
- return anId;
- }
- }
- }
- }
- return null;
+ Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>> connEdge = connectorActivityMap.get(cdId);
+ return connEdge.getLeft().getLeft().getActivityId();
}
- public RecordDescriptor getActivityInputRecordDescriptor(ActivityId hanId, int inputIndex) {
- int opInputIndex = getActivityInputMap().get(hanId).get(inputIndex);
- return jobSpec.getOperatorInputRecordDescriptor(hanId.getOperatorDescriptorId(), opInputIndex);
+ public IConnectorPolicyAssignmentPolicy getConnectorPolicyAssignmentPolicy() {
+ return connectorPolicyAssignmentPolicy;
}
- public RecordDescriptor getActivityOutputRecordDescriptor(ActivityId hanId, int outputIndex) {
- int opOutputIndex = getActivityOutputMap().get(hanId).get(outputIndex);
- return jobSpec.getOperatorOutputRecordDescriptor(hanId.getOperatorDescriptorId(), opOutputIndex);
+ public void setConnectorPolicyAssignmentPolicy(IConnectorPolicyAssignmentPolicy connectorPolicyAssignmentPolicy) {
+ this.connectorPolicyAssignmentPolicy = connectorPolicyAssignmentPolicy;
+ }
+
+ public void setMaxReattempts(int maxReattempts) {
+ this.maxReattempts = maxReattempts;
+ }
+
+ public int getMaxReattempts() {
+ return maxReattempts;
+ }
+
+ public IJobletEventListenerFactory getJobletEventListenerFactory() {
+ return jobletEventListenerFactory;
+ }
+
+ public void setJobletEventListenerFactory(IJobletEventListenerFactory jobletEventListenerFactory) {
+ this.jobletEventListenerFactory = jobletEventListenerFactory;
+ }
+
+ public IGlobalJobDataFactory getGlobalJobDataFactory() {
+ return globalJobDataFactory;
+ }
+
+ public void setGlobalJobDataFactory(IGlobalJobDataFactory globalJobDataFactory) {
+ this.globalJobDataFactory = globalJobDataFactory;
}
@Override
public String toString() {
StringBuilder buffer = new StringBuilder();
- buffer.append("ActivityNodes: " + activityNodes);
+ buffer.append("ActivityNodes: " + activityMap);
buffer.append('\n');
buffer.append("Blocker->Blocked: " + blocker2blockedMap);
buffer.append('\n');
@@ -212,13 +177,12 @@
jplan.put("flags", jobFlags.toString());
JSONArray jans = new JSONArray();
- for (IActivity an : activityNodes.values()) {
+ for (IActivity an : activityMap.values()) {
JSONObject jan = new JSONObject();
jan.put("id", an.getActivityId().toString());
jan.put("java-class", an.getClass().getName());
- jan.put("operator-id", an.getActivityId().getOperatorDescriptorId().toString());
- List<IConnectorDescriptor> inputs = getActivityInputConnectorDescriptors(an.getActivityId());
+ List<IConnectorDescriptor> inputs = activityInputMap.get(an.getActivityId());
if (inputs != null) {
JSONArray jInputs = new JSONArray();
for (int i = 0; i < inputs.size(); ++i) {
@@ -230,7 +194,7 @@
jan.put("inputs", jInputs);
}
- List<IConnectorDescriptor> outputs = getActivityOutputConnectorDescriptors(an.getActivityId());
+ List<IConnectorDescriptor> outputs = activityOutputMap.get(an.getActivityId());
if (outputs != null) {
JSONArray jOutputs = new JSONArray();
for (int i = 0; i < outputs.size(); ++i) {
diff --git a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
index dd0e39e..a9a69fa 100644
--- a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
@@ -46,7 +46,6 @@
import edu.uci.ics.hyracks.control.cc.work.GetIpAddressNodeNameMapWork;
import edu.uci.ics.hyracks.control.cc.work.GetJobStatusWork;
import edu.uci.ics.hyracks.control.cc.work.GetNodeControllersInfoWork;
-import edu.uci.ics.hyracks.control.cc.work.JobCreateWork;
import edu.uci.ics.hyracks.control.cc.work.JobStartWork;
import edu.uci.ics.hyracks.control.cc.work.JobletCleanupNotificationWork;
import edu.uci.ics.hyracks.control.cc.work.NodeHeartbeatWork;
@@ -282,14 +281,6 @@
return;
}
- case CREATE_JOB: {
- HyracksClientInterfaceFunctions.CreateJobFunction cjf = (HyracksClientInterfaceFunctions.CreateJobFunction) fn;
- JobId jobId = createJobId();
- workQueue.schedule(new JobCreateWork(ClusterControllerService.this, jobId, cjf.getAppName(), cjf
- .getJobSpec(), cjf.getJobFlags(), new IPCResponder<JobId>(handle, mid)));
- return;
- }
-
case GET_JOB_STATUS: {
HyracksClientInterfaceFunctions.GetJobStatusFunction gjsf = (HyracksClientInterfaceFunctions.GetJobStatusFunction) fn;
workQueue.schedule(new GetJobStatusWork(ClusterControllerService.this, gjsf.getJobId(),
@@ -299,8 +290,9 @@
case START_JOB: {
HyracksClientInterfaceFunctions.StartJobFunction sjf = (HyracksClientInterfaceFunctions.StartJobFunction) fn;
- workQueue.schedule(new JobStartWork(ClusterControllerService.this, sjf.getJobId(),
- new IPCResponder<Object>(handle, mid)));
+ JobId jobId = createJobId();
+ workQueue.schedule(new JobStartWork(ClusterControllerService.this, sjf.getAppName(), sjf
+ .getJobSpec(), sjf.getJobFlags(), jobId, new IPCResponder<JobId>(handle, mid)));
return;
}
diff --git a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.java b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.java
index 1d96a71..a486c33 100644
--- a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.java
+++ b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/adminconsole/pages/JobDetailsPage.java
@@ -35,7 +35,6 @@
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.work.GetJobActivityGraphJSONWork;
import edu.uci.ics.hyracks.control.cc.work.GetJobRunJSONWork;
-import edu.uci.ics.hyracks.control.cc.work.GetJobSpecificationJSONWork;
public class JobDetailsPage extends AbstractPage {
private static final long serialVersionUID = 1L;
@@ -49,12 +48,6 @@
JobId jobId = JobId.parse(jobIdStr.toString());
- GetJobSpecificationJSONWork gjsw = new GetJobSpecificationJSONWork(ccs, jobId);
- ccs.getWorkQueue().scheduleAndSync(gjsw);
- Label jobspec = new Label("job-specification", gjsw.getJSON().toString());
- jobspec.setEscapeModelStrings(false);
- add(jobspec);
-
GetJobActivityGraphJSONWork gjagw = new GetJobActivityGraphJSONWork(ccs, jobId);
ccs.getWorkQueue().scheduleAndSync(gjagw);
Label jag = new Label("job-activity-graph", gjagw.getJSON().toString());
diff --git a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
index fbf0b11..325ea84 100644
--- a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
+++ b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/application/CCApplicationContext.java
@@ -26,11 +26,10 @@
import edu.uci.ics.hyracks.api.context.ICCContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.IJobLifecycleListener;
-import edu.uci.ics.hyracks.api.job.IJobSpecificationFactory;
import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.control.cc.job.DeserializingJobSpecificationFactory;
+import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
import edu.uci.ics.hyracks.control.common.application.ApplicationContext;
import edu.uci.ics.hyracks.control.common.context.ServerContext;
import edu.uci.ics.hyracks.control.common.work.IResultCallback;
@@ -44,8 +43,6 @@
protected IResultCallback<Object> initializationCallback;
protected IResultCallback<Object> deinitializationCallback;
- private IJobSpecificationFactory jobSpecFactory;
-
private List<IJobLifecycleListener> jobLifecycleListeners;
public CCApplicationContext(ServerContext serverCtx, ICCContext ccContext, String appName) throws IOException {
@@ -53,7 +50,6 @@
this.ccContext = ccContext;
initPendingNodeIds = new HashSet<String>();
deinitPendingNodeIds = new HashSet<String>();
- jobSpecFactory = DeserializingJobSpecificationFactory.INSTANCE;
jobLifecycleListeners = new ArrayList<IJobLifecycleListener>();
}
@@ -67,13 +63,14 @@
return ccContext;
}
- @Override
- public void setJobSpecificationFactory(IJobSpecificationFactory jobSpecFactory) {
- this.jobSpecFactory = jobSpecFactory;
- }
-
public JobSpecification createJobSpecification(byte[] bytes) throws HyracksException {
- return jobSpecFactory.createJobSpecification(bytes, (ICCBootstrap) bootstrap, this);
+ try {
+ return (JobSpecification) JavaSerializationUtils.deserialize(bytes, getClassLoader());
+ } catch (IOException e) {
+ throw new HyracksException(e);
+ } catch (ClassNotFoundException e) {
+ throw new HyracksException(e);
+ }
}
@Override
@@ -105,7 +102,8 @@
}
}
- public synchronized void notifyJobCreation(JobId jobId, IOperatorDescriptorRegistry specification) throws HyracksException {
+ public synchronized void notifyJobCreation(JobId jobId, IOperatorDescriptorRegistry specification)
+ throws HyracksException {
for (IJobLifecycleListener l : jobLifecycleListeners) {
l.notifyJobCreation(jobId, specification);
}
diff --git a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityCluster.java b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityCluster.java
index f5b74cf..d351d4a 100644
--- a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityCluster.java
+++ b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/ActivityCluster.java
@@ -71,4 +71,9 @@
public void setPlan(ActivityClusterPlan acp) {
this.acp = acp;
}
+
+ @Override
+ public String toString() {
+ return String.valueOf(activities);
+ }
}
\ No newline at end of file
diff --git a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/DeserializingJobSpecificationFactory.java b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/DeserializingJobSpecificationFactory.java
deleted file mode 100644
index c746f34..0000000
--- a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/DeserializingJobSpecificationFactory.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.job;
-
-import java.io.IOException;
-
-import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
-import edu.uci.ics.hyracks.api.application.ICCBootstrap;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.api.job.IJobSpecificationFactory;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
-
-public class DeserializingJobSpecificationFactory implements IJobSpecificationFactory {
- public static final IJobSpecificationFactory INSTANCE = new DeserializingJobSpecificationFactory();
-
- private DeserializingJobSpecificationFactory() {
- }
-
- @Override
- public JobSpecification createJobSpecification(byte[] bytes, ICCBootstrap bootstrap, ICCApplicationContext appCtx)
- throws HyracksException {
- try {
- return (JobSpecification) JavaSerializationUtils.deserialize(bytes, appCtx.getClassLoader());
- } catch (IOException e) {
- throw new HyracksException(e);
- } catch (ClassNotFoundException e) {
- throw new HyracksException(e);
- }
- }
-}
\ No newline at end of file
diff --git a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobActivityGraphBuilder.java b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobActivityGraphBuilder.java
index 6effd84..a7b60e4 100644
--- a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobActivityGraphBuilder.java
+++ b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/job/JobActivityGraphBuilder.java
@@ -2,6 +2,7 @@
import java.util.ArrayList;
import java.util.EnumSet;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -9,9 +10,14 @@
import java.util.logging.Level;
import java.util.logging.Logger;
+import org.apache.commons.lang3.tuple.Pair;
+
import edu.uci.ics.hyracks.api.dataflow.ActivityId;
+import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
import edu.uci.ics.hyracks.api.dataflow.IActivity;
import edu.uci.ics.hyracks.api.dataflow.IActivityGraphBuilder;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.job.JobActivityGraph;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobSpecification;
@@ -19,7 +25,32 @@
public class JobActivityGraphBuilder implements IActivityGraphBuilder {
private static final Logger LOGGER = Logger.getLogger(JobActivityGraphBuilder.class.getName());
- private JobActivityGraph jag;
+ private final Map<ActivityId, IOperatorDescriptor> activityOperatorMap;
+
+ private final JobActivityGraph jag;
+
+ private final JobSpecification jobSpec;
+
+ private final Map<ConnectorDescriptorId, Pair<IActivity, Integer>> connectorProducerMap;
+
+ private final Map<ConnectorDescriptorId, Pair<IActivity, Integer>> connectorConsumerMap;
+
+ public JobActivityGraphBuilder(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) {
+ activityOperatorMap = new HashMap<ActivityId, IOperatorDescriptor>();
+ jag = new JobActivityGraph(appName, jobFlags);
+ this.jobSpec = jobSpec;
+ jag.setConnectorPolicyAssignmentPolicy(jobSpec.getConnectorPolicyAssignmentPolicy());
+ jag.setGlobalJobDataFactory(jobSpec.getGlobalJobDataFactory());
+ jag.setJobletEventListenerFactory(jobSpec.getJobletEventListenerFactory());
+ jag.setMaxReattempts(jobSpec.getMaxReattempts());
+ connectorProducerMap = new HashMap<ConnectorDescriptorId, Pair<IActivity, Integer>>();
+ connectorConsumerMap = new HashMap<ConnectorDescriptorId, Pair<IActivity, Integer>>();
+ }
+
+ public void addConnector(IConnectorDescriptor conn) {
+ jag.getConnectorMap().put(conn.getConnectorId(), conn);
+ jag.getConnectorRecordDescriptorMap().put(conn.getConnectorId(), jobSpec.getConnectorRecordDescriptor(conn));
+ }
@Override
public void addBlockingEdge(IActivity blocker, IActivity blocked) {
@@ -33,9 +64,10 @@
LOGGER.finest("Adding source edge: " + task.getActivityId().getOperatorDescriptorId() + ":"
+ operatorInputIndex + " -> " + task.getActivityId() + ":" + taskInputIndex);
}
- insertIntoIndexedMap(jag.getActivityInputMap(), task.getActivityId(), taskInputIndex, operatorInputIndex);
- insertIntoIndexedMap(jag.getOperatorInputMap(), task.getActivityId().getOperatorDescriptorId(),
- operatorInputIndex, task.getActivityId());
+ IOperatorDescriptor op = activityOperatorMap.get(task.getActivityId());
+ IConnectorDescriptor conn = jobSpec.getInputConnectorDescriptor(op, operatorInputIndex);
+ insertIntoIndexedMap(jag.getActivityInputMap(), task.getActivityId(), taskInputIndex, conn);
+ connectorConsumerMap.put(conn.getConnectorId(), Pair.of(task, taskInputIndex));
}
@Override
@@ -44,16 +76,28 @@
LOGGER.finest("Adding target edge: " + task.getActivityId().getOperatorDescriptorId() + ":"
+ operatorOutputIndex + " -> " + task.getActivityId() + ":" + taskOutputIndex);
}
- insertIntoIndexedMap(jag.getActivityOutputMap(), task.getActivityId(), taskOutputIndex, operatorOutputIndex);
- insertIntoIndexedMap(jag.getOperatorOutputMap(), task.getActivityId().getOperatorDescriptorId(),
- operatorOutputIndex, task.getActivityId());
+ IOperatorDescriptor op = activityOperatorMap.get(task.getActivityId());
+ IConnectorDescriptor conn = jobSpec.getOutputConnectorDescriptor(op, operatorOutputIndex);
+ insertIntoIndexedMap(jag.getActivityOutputMap(), task.getActivityId(), taskOutputIndex, conn);
+ connectorProducerMap.put(conn.getConnectorId(), Pair.of(task, taskOutputIndex));
}
@Override
- public void addActivity(IActivity task) {
+ public void addActivity(IOperatorDescriptor op, IActivity task) {
+ activityOperatorMap.put(task.getActivityId(), op);
ActivityId activityId = task.getActivityId();
- jag.getActivityNodeMap().put(activityId, task);
- addToValueSet(jag.getOperatorActivityMap(), activityId.getOperatorDescriptorId(), activityId);
+ jag.getActivityMap().put(activityId, task);
+ }
+
+ public void finish() {
+ Map<ConnectorDescriptorId, Pair<Pair<IActivity, Integer>, Pair<IActivity, Integer>>> caMap = jag
+ .getConnectorActivityMap();
+ for (Map.Entry<ConnectorDescriptorId, Pair<IActivity, Integer>> e : connectorProducerMap.entrySet()) {
+ ConnectorDescriptorId cdId = e.getKey();
+ Pair<IActivity, Integer> producer = e.getValue();
+ Pair<IActivity, Integer> consumer = connectorConsumerMap.get(cdId);
+ caMap.put(cdId, Pair.of(producer, consumer));
+ }
}
private <K, V> void addToValueSet(Map<K, Set<V>> map, K n1, V n2) {
@@ -72,10 +116,6 @@
}
}
- public void init(String appName, JobSpecification jobSpec, EnumSet<JobFlag> jobFlags) {
- jag = new JobActivityGraph(appName, jobSpec, jobFlags);
- }
-
private <K, V> void insertIntoIndexedMap(Map<K, List<V>> map, K key, int index, V value) {
List<V> vList = map.get(key);
if (vList == null) {
diff --git a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.java b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.java
index 9d6aa71..9acc4b8 100644
--- a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.java
+++ b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterGraphBuilder.java
@@ -25,11 +25,8 @@
import org.apache.commons.lang3.tuple.Pair;
import edu.uci.ics.hyracks.api.dataflow.ActivityId;
-import edu.uci.ics.hyracks.api.dataflow.IActivity;
import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
import edu.uci.ics.hyracks.api.job.JobActivityGraph;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
import edu.uci.ics.hyracks.control.cc.job.ActivityClusterId;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
@@ -43,33 +40,22 @@
this.jobRun = jobRun;
}
- private static Pair<ActivityId, ActivityId> findMergePair(JobActivityGraph jag, JobSpecification spec,
- Set<ActivityCluster> eqSets) {
- Map<ActivityId, IActivity> activityNodeMap = jag.getActivityNodeMap();
+ private static Pair<ActivityId, ActivityId> findMergePair(JobActivityGraph jag, Set<ActivityCluster> eqSets) {
for (ActivityCluster eqSet : eqSets) {
for (ActivityId t : eqSet.getActivities()) {
- IActivity activity = activityNodeMap.get(t);
- List<Integer> inputList = jag.getActivityInputMap().get(t);
+ List<IConnectorDescriptor> inputList = jag.getActivityInputMap().get(t);
if (inputList != null) {
- for (Integer idx : inputList) {
- IConnectorDescriptor conn = spec.getInputConnectorDescriptor(activity.getActivityId()
- .getOperatorDescriptorId(), idx);
- OperatorDescriptorId producerId = spec.getProducer(conn).getOperatorId();
- int producerOutputIndex = spec.getProducerOutputIndex(conn);
- ActivityId inTask = jag.getOperatorOutputMap().get(producerId).get(producerOutputIndex);
+ for (IConnectorDescriptor conn : inputList) {
+ ActivityId inTask = jag.getProducerActivity(conn.getConnectorId());
if (!eqSet.getActivities().contains(inTask)) {
return Pair.<ActivityId, ActivityId> of(t, inTask);
}
}
}
- List<Integer> outputList = jag.getActivityOutputMap().get(t);
+ List<IConnectorDescriptor> outputList = jag.getActivityOutputMap().get(t);
if (outputList != null) {
- for (Integer idx : outputList) {
- IConnectorDescriptor conn = spec.getOutputConnectorDescriptor(activity.getActivityId()
- .getOperatorDescriptorId(), idx);
- OperatorDescriptorId consumerId = spec.getConsumer(conn).getOperatorId();
- int consumerInputIndex = spec.getConsumerInputIndex(conn);
- ActivityId outTask = jag.getOperatorInputMap().get(consumerId).get(consumerInputIndex);
+ for (IConnectorDescriptor conn : outputList) {
+ ActivityId outTask = jag.getConsumerActivity(conn.getConnectorId());
if (!eqSet.getActivities().contains(outTask)) {
return Pair.<ActivityId, ActivityId> of(t, outTask);
}
@@ -81,27 +67,23 @@
}
public Set<ActivityCluster> inferActivityClusters(JobActivityGraph jag) {
- JobSpecification spec = jag.getJobSpecification();
-
/*
* Build initial equivalence sets map. We create a map such that for each IOperatorTask, t -> { t }
*/
Map<ActivityId, ActivityCluster> stageMap = new HashMap<ActivityId, ActivityCluster>();
Set<ActivityCluster> stages = new HashSet<ActivityCluster>();
- for (Set<ActivityId> taskIds : jag.getOperatorActivityMap().values()) {
- for (ActivityId taskId : taskIds) {
- Set<ActivityId> eqSet = new HashSet<ActivityId>();
- eqSet.add(taskId);
- ActivityCluster stage = new ActivityCluster(eqSet);
- stageMap.put(taskId, stage);
- stages.add(stage);
- }
+ for (ActivityId taskId : jag.getActivityMap().keySet()) {
+ Set<ActivityId> eqSet = new HashSet<ActivityId>();
+ eqSet.add(taskId);
+ ActivityCluster stage = new ActivityCluster(eqSet);
+ stageMap.put(taskId, stage);
+ stages.add(stage);
}
boolean changed = true;
while (changed) {
changed = false;
- Pair<ActivityId, ActivityId> pair = findMergePair(jag, spec, stages);
+ Pair<ActivityId, ActivityId> pair = findMergePair(jag, stages);
if (pair != null) {
merge(stageMap, stages, pair.getLeft(), pair.getRight());
changed = true;
diff --git a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
index d04bf81..23f53f4 100644
--- a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
+++ b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/ActivityClusterPlanner.java
@@ -184,7 +184,7 @@
for (ActivityId ac1 : activities) {
Task[] ac1TaskStates = activityPlanMap.get(ac1).getTasks();
int nProducers = ac1TaskStates.length;
- List<IConnectorDescriptor> outputConns = jag.getActivityOutputConnectorDescriptors(ac1);
+ List<IConnectorDescriptor> outputConns = jag.getActivityOutputMap().get(ac1);
if (outputConns != null) {
for (IConnectorDescriptor c : outputConns) {
ConnectorDescriptorId cdId = c.getConnectorId();
@@ -334,7 +334,7 @@
for (ActivityId a1 : activities) {
Task[] ac1TaskStates = taskMap.get(a1).getTasks();
int nProducers = ac1TaskStates.length;
- List<IConnectorDescriptor> outputConns = jag.getActivityOutputConnectorDescriptors(a1);
+ List<IConnectorDescriptor> outputConns = jag.getActivityOutputMap().get(a1);
if (outputConns != null) {
for (IConnectorDescriptor c : outputConns) {
ConnectorDescriptorId cdId = c.getConnectorId();
@@ -356,7 +356,7 @@
}
private IConnectorPolicy assignConnectorPolicy(IConnectorDescriptor c, int nProducers, int nConsumers, int[] fanouts) {
- IConnectorPolicyAssignmentPolicy cpap = scheduler.getJobRun().getJobActivityGraph().getJobSpecification()
+ IConnectorPolicyAssignmentPolicy cpap = scheduler.getJobRun().getJobActivityGraph()
.getConnectorPolicyAssignmentPolicy();
if (cpap != null) {
return cpap.getConnectorPolicyAssignment(c, nProducers, nConsumers, fanouts);
@@ -368,6 +368,7 @@
throws HyracksException {
PartitionConstraintSolver solver = scheduler.getSolver();
JobRun jobRun = scheduler.getJobRun();
+ JobActivityGraph jag = jobRun.getJobActivityGraph();
Set<LValueConstraintExpression> lValues = new HashSet<LValueConstraintExpression>();
for (ActivityId anId : ac.getActivities()) {
lValues.add(new PartitionCountExpression(anId.getOperatorDescriptorId()));
@@ -393,22 +394,25 @@
for (ActivityId anId : ac.getActivities()) {
int nParts = nPartMap.get(anId.getOperatorDescriptorId());
int[] nInputPartitions = null;
- List<IConnectorDescriptor> inputs = jobRun.getJobActivityGraph().getActivityInputConnectorDescriptors(anId);
+ List<IConnectorDescriptor> inputs = jag.getActivityInputMap().get(anId);
if (inputs != null) {
nInputPartitions = new int[inputs.size()];
for (int i = 0; i < nInputPartitions.length; ++i) {
- nInputPartitions[i] = nPartMap.get(jobRun.getJobActivityGraph()
- .getProducerActivity(inputs.get(i).getConnectorId()).getOperatorDescriptorId());
+ ConnectorDescriptorId cdId = inputs.get(i).getConnectorId();
+ ActivityId aid = jag.getProducerActivity(cdId);
+ Integer nPartInt = nPartMap.get(aid.getOperatorDescriptorId());
+ nInputPartitions[i] = nPartInt;
}
}
int[] nOutputPartitions = null;
- List<IConnectorDescriptor> outputs = jobRun.getJobActivityGraph().getActivityOutputConnectorDescriptors(
- anId);
+ List<IConnectorDescriptor> outputs = jag.getActivityOutputMap().get(anId);
if (outputs != null) {
nOutputPartitions = new int[outputs.size()];
for (int i = 0; i < nOutputPartitions.length; ++i) {
- nOutputPartitions[i] = nPartMap.get(jobRun.getJobActivityGraph()
- .getConsumerActivity(outputs.get(i).getConnectorId()).getOperatorDescriptorId());
+ ConnectorDescriptorId cdId = outputs.get(i).getConnectorId();
+ ActivityId aid = jag.getConsumerActivity(cdId);
+ Integer nPartInt = nPartMap.get(aid.getOperatorDescriptorId());
+ nOutputPartitions[i] = nPartInt;
}
}
ActivityPartitionDetails apd = new ActivityPartitionDetails(nParts, nInputPartitions, nOutputPartitions);
diff --git a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
index 4a315ab..b3a76ad 100644
--- a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
+++ b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/scheduler/JobScheduler.java
@@ -15,6 +15,7 @@
package edu.uci.ics.hyracks.control.cc.scheduler;
import java.util.ArrayList;
+import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@@ -25,32 +26,24 @@
import java.util.logging.Level;
import java.util.logging.Logger;
-import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
import edu.uci.ics.hyracks.api.constraints.Constraint;
-import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
import edu.uci.ics.hyracks.api.constraints.expressions.LValueConstraintExpression;
import edu.uci.ics.hyracks.api.constraints.expressions.PartitionLocationExpression;
import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
-import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
import edu.uci.ics.hyracks.api.dataflow.TaskId;
import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.JobActivityGraph;
import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
import edu.uci.ics.hyracks.api.util.JavaSerializationUtils;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.NodeControllerState;
import edu.uci.ics.hyracks.control.cc.job.ActivityCluster;
-import edu.uci.ics.hyracks.control.cc.job.IConnectorDescriptorVisitor;
-import edu.uci.ics.hyracks.control.cc.job.IOperatorDescriptorVisitor;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.job.PlanUtils;
import edu.uci.ics.hyracks.control.cc.job.Task;
import edu.uci.ics.hyracks.control.cc.job.TaskAttempt;
import edu.uci.ics.hyracks.control.cc.job.TaskCluster;
@@ -75,12 +68,15 @@
private Set<ActivityCluster> rootActivityClusters;
- public JobScheduler(ClusterControllerService ccs, JobRun jobRun) {
+ public JobScheduler(ClusterControllerService ccs, JobRun jobRun, Collection<Constraint> constraints) {
this.ccs = ccs;
this.jobRun = jobRun;
solver = new PartitionConstraintSolver();
partitionProducingTaskClusterMap = new HashMap<PartitionId, TaskCluster>();
inProgressTaskClusters = new HashSet<TaskCluster>();
+ solver.addConstraints(constraints);
+ ActivityClusterGraphBuilder acgb = new ActivityClusterGraphBuilder(jobRun);
+ rootActivityClusters = acgb.inferActivityClusters(jobRun.getJobActivityGraph());
}
public JobRun getJobRun() {
@@ -92,43 +88,9 @@
}
public void startJob() throws HyracksException {
- if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("Starting Job: " + jobRun.getJobActivityGraph().getJobSpecification());
- }
- analyze();
startRunnableActivityClusters();
}
- private void analyze() throws HyracksException {
- final JobActivityGraph jag = jobRun.getJobActivityGraph();
- final ICCApplicationContext appCtx = ccs.getApplicationMap().get(jag.getApplicationName());
- JobSpecification spec = jag.getJobSpecification();
- final Set<Constraint> contributedConstraints = new HashSet<Constraint>();
- final IConstraintAcceptor acceptor = new IConstraintAcceptor() {
- @Override
- public void addConstraint(Constraint constraint) {
- contributedConstraints.add(constraint);
- }
- };
- PlanUtils.visit(spec, new IOperatorDescriptorVisitor() {
- @Override
- public void visit(IOperatorDescriptor op) {
- op.contributeSchedulingConstraints(acceptor, jag, appCtx);
- }
- });
- PlanUtils.visit(spec, new IConnectorDescriptorVisitor() {
- @Override
- public void visit(IConnectorDescriptor conn) {
- conn.contributeSchedulingConstraints(acceptor, jag, appCtx);
- }
- });
- contributedConstraints.addAll(spec.getUserConstraints());
- solver.addConstraints(contributedConstraints);
-
- ActivityClusterGraphBuilder acgb = new ActivityClusterGraphBuilder(jobRun);
- rootActivityClusters = acgb.inferActivityClusters(jag);
- }
-
private void findRunnableTaskClusterRoots(Set<TaskCluster> frontier, Set<ActivityCluster> roots)
throws HyracksException {
for (ActivityCluster root : roots) {
@@ -644,7 +606,7 @@
lastAttempt.setStatus(TaskClusterAttempt.TaskClusterStatus.FAILED);
lastAttempt.setEndTime(System.currentTimeMillis());
abortDoomedTaskClusters();
- if (lastAttempt.getAttempt() >= jobRun.getJobActivityGraph().getJobSpecification().getMaxReattempts()) {
+ if (lastAttempt.getAttempt() >= jobRun.getJobActivityGraph().getMaxReattempts()) {
abortJob(new HyracksException(details));
return;
}
diff --git a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/JobsRESTAPIFunction.java b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/JobsRESTAPIFunction.java
index 4135b18..c964e4e 100644
--- a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/JobsRESTAPIFunction.java
+++ b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/web/JobsRESTAPIFunction.java
@@ -21,7 +21,6 @@
import edu.uci.ics.hyracks.control.cc.web.util.IJSONOutputFunction;
import edu.uci.ics.hyracks.control.cc.work.GetJobActivityGraphJSONWork;
import edu.uci.ics.hyracks.control.cc.work.GetJobRunJSONWork;
-import edu.uci.ics.hyracks.control.cc.work.GetJobSpecificationJSONWork;
import edu.uci.ics.hyracks.control.cc.work.GetJobSummariesJSONWork;
public class JobsRESTAPIFunction implements IJSONOutputFunction {
@@ -49,11 +48,7 @@
case 2: {
JobId jobId = JobId.parse(arguments[0]);
- if ("job-specification".equalsIgnoreCase(arguments[1])) {
- GetJobSpecificationJSONWork gjse = new GetJobSpecificationJSONWork(ccs, jobId);
- ccs.getWorkQueue().scheduleAndSync(gjse);
- result.put("result", gjse.getJSON());
- } else if ("job-activity-graph".equalsIgnoreCase(arguments[1])) {
+ if ("job-activity-graph".equalsIgnoreCase(arguments[1])) {
GetJobActivityGraphJSONWork gjage = new GetJobActivityGraphJSONWork(ccs, jobId);
ccs.getWorkQueue().scheduleAndSync(gjage);
result.put("result", gjage.getJSON());
diff --git a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobSpecificationJSONWork.java b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobSpecificationJSONWork.java
deleted file mode 100644
index 05b7cae..0000000
--- a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetJobSpecificationJSONWork.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.work;
-
-import org.json.JSONObject;
-
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
-
-public class GetJobSpecificationJSONWork extends SynchronizableWork {
- private final ClusterControllerService ccs;
- private final JobId jobId;
- private JSONObject json;
-
- public GetJobSpecificationJSONWork(ClusterControllerService ccs, JobId jobId) {
- this.ccs = ccs;
- this.jobId = jobId;
- }
-
- @Override
- protected void doRun() throws Exception {
- JobRun run = ccs.getActiveRunMap().get(jobId);
- if (run == null) {
- run = ccs.getRunMapArchive().get(jobId);
- if (run == null) {
- json = new JSONObject();
- return;
- }
- }
- json = run.getJobActivityGraph().getJobSpecification().toJSON();
- }
-
- public JSONObject getJSON() {
- return json;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
index 07088bb..8d8e85d 100644
--- a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
+++ b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCleanupWork.java
@@ -17,10 +17,16 @@
import java.util.Set;
import java.util.logging.Logger;
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.cc.NodeControllerState;
+import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
import edu.uci.ics.hyracks.control.common.work.AbstractWork;
@@ -53,13 +59,44 @@
Set<String> targetNodes = run.getParticipatingNodeIds();
run.getCleanupPendingNodeIds().addAll(targetNodes);
run.setPendingStatus(status, exception);
- for (String n : targetNodes) {
- NodeControllerState ncs = ccs.getNodeMap().get(n);
+ if (targetNodes != null && !targetNodes.isEmpty()) {
+ for (String n : targetNodes) {
+ NodeControllerState ncs = ccs.getNodeMap().get(n);
+ try {
+ ncs.getNodeController().cleanUpJoblet(jobId, status);
+ } catch (Exception e) {
+ e.printStackTrace();
+ }
+ }
+ } else {
+ CCApplicationContext appCtx = ccs.getApplicationMap().get(run.getJobActivityGraph().getApplicationName());
+ if (appCtx != null) {
+ try {
+ appCtx.notifyJobFinish(jobId);
+ } catch (HyracksException e) {
+ e.printStackTrace();
+ }
+ }
+ run.setStatus(run.getPendingStatus(), run.getPendingException());
+ ccs.getActiveRunMap().remove(jobId);
+ ccs.getRunMapArchive().put(jobId, run);
try {
- ncs.getNodeController().cleanUpJoblet(jobId, status);
+ ccs.getJobLogFile().log(createJobLogObject(run));
} catch (Exception e) {
- e.printStackTrace();
+ throw new RuntimeException(e);
}
}
}
+
+ private JSONObject createJobLogObject(final JobRun run) {
+ JSONObject jobLogObject = new JSONObject();
+ try {
+ JobActivityGraph jag = run.getJobActivityGraph();
+ jobLogObject.put("job-activity-graph", jag.toJSON());
+ jobLogObject.put("job-run", run.toJSON());
+ } catch (JSONException e) {
+ throw new RuntimeException(e);
+ }
+ return jobLogObject;
+ }
}
\ No newline at end of file
diff --git a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCreateWork.java b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCreateWork.java
deleted file mode 100644
index b7cf629..0000000
--- a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobCreateWork.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.hyracks.control.cc.work;
-
-import java.util.EnumSet;
-
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-import edu.uci.ics.hyracks.api.job.JobActivityGraph;
-import edu.uci.ics.hyracks.api.job.JobFlag;
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.api.job.JobStatus;
-import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
-import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
-import edu.uci.ics.hyracks.control.cc.job.IOperatorDescriptorVisitor;
-import edu.uci.ics.hyracks.control.cc.job.JobActivityGraphBuilder;
-import edu.uci.ics.hyracks.control.cc.job.JobRun;
-import edu.uci.ics.hyracks.control.cc.job.PlanUtils;
-import edu.uci.ics.hyracks.control.cc.scheduler.JobScheduler;
-import edu.uci.ics.hyracks.control.common.work.IResultCallback;
-import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
-
-public class JobCreateWork extends SynchronizableWork {
- private final ClusterControllerService ccs;
- private final byte[] jobSpec;
- private final EnumSet<JobFlag> jobFlags;
- private final JobId jobId;
- private final String appName;
- private final IResultCallback<JobId> callback;
-
- public JobCreateWork(ClusterControllerService ccs, JobId jobId, String appName, byte[] jobSpec,
- EnumSet<JobFlag> jobFlags, IResultCallback<JobId> callback) {
- this.jobId = jobId;
- this.ccs = ccs;
- this.jobSpec = jobSpec;
- this.jobFlags = jobFlags;
- this.appName = appName;
- this.callback = callback;
- }
-
- @Override
- protected void doRun() throws Exception {
- try {
- CCApplicationContext appCtx = ccs.getApplicationMap().get(appName);
- if (appCtx == null) {
- throw new HyracksException("No application with id " + appName + " found");
- }
- JobSpecification spec = appCtx.createJobSpecification(jobSpec);
-
- final JobActivityGraphBuilder builder = new JobActivityGraphBuilder();
- builder.init(appName, spec, jobFlags);
- PlanUtils.visit(spec, new IOperatorDescriptorVisitor() {
- @Override
- public void visit(IOperatorDescriptor op) {
- op.contributeActivities(builder);
- }
- });
- final JobActivityGraph jag = builder.getActivityGraph();
-
- JobRun run = new JobRun(jobId, jag);
-
- run.setStatus(JobStatus.INITIALIZED, null);
-
- ccs.getActiveRunMap().put(jobId, run);
- JobScheduler jrs = new JobScheduler(ccs, run);
- run.setScheduler(jrs);
- appCtx.notifyJobCreation(jobId, spec);
- callback.setValue(jobId);
- } catch (Exception e) {
- callback.setException(e);
- return;
- }
- }
-
- public JobId getJobId() {
- return jobId;
- }
-}
\ No newline at end of file
diff --git a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
index dd834ed..fa22691 100644
--- a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
+++ b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
@@ -14,41 +14,110 @@
*/
package edu.uci.ics.hyracks.control.cc.work;
+import java.util.EnumSet;
+import java.util.HashSet;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.constraints.Constraint;
+import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobActivityGraph;
+import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
+import edu.uci.ics.hyracks.control.cc.job.IConnectorDescriptorVisitor;
+import edu.uci.ics.hyracks.control.cc.job.IOperatorDescriptorVisitor;
+import edu.uci.ics.hyracks.control.cc.job.JobActivityGraphBuilder;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
+import edu.uci.ics.hyracks.control.cc.job.PlanUtils;
+import edu.uci.ics.hyracks.control.cc.scheduler.JobScheduler;
import edu.uci.ics.hyracks.control.common.work.IResultCallback;
import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
public class JobStartWork extends SynchronizableWork {
private final ClusterControllerService ccs;
+ private final byte[] jobSpec;
+ private final EnumSet<JobFlag> jobFlags;
private final JobId jobId;
- private final IResultCallback<Object> callback;
+ private final String appName;
+ private final IResultCallback<JobId> callback;
- public JobStartWork(ClusterControllerService ccs, JobId jobId, IResultCallback<Object> callback) {
- this.ccs = ccs;
+ public JobStartWork(ClusterControllerService ccs, String appName, byte[] jobSpec, EnumSet<JobFlag> jobFlags,
+ JobId jobId, IResultCallback<JobId> callback) {
this.jobId = jobId;
+ this.ccs = ccs;
+ this.jobSpec = jobSpec;
+ this.jobFlags = jobFlags;
+ this.appName = appName;
this.callback = callback;
}
@Override
protected void doRun() throws Exception {
try {
- JobRun run = ccs.getActiveRunMap().get(jobId);
- if (run == null) {
- throw new Exception("Unable to find job with id = " + jobId);
+ final CCApplicationContext appCtx = ccs.getApplicationMap().get(appName);
+ if (appCtx == null) {
+ throw new HyracksException("No application with id " + appName + " found");
}
- if (run.getStatus() != JobStatus.INITIALIZED) {
- throw new Exception("Job already started");
- }
+ JobSpecification spec = appCtx.createJobSpecification(jobSpec);
+
+ final JobActivityGraphBuilder builder = new JobActivityGraphBuilder(appName, spec, jobFlags);
+ PlanUtils.visit(spec, new IConnectorDescriptorVisitor() {
+ @Override
+ public void visit(IConnectorDescriptor conn) throws HyracksException {
+ builder.addConnector(conn);
+ }
+ });
+ PlanUtils.visit(spec, new IOperatorDescriptorVisitor() {
+ @Override
+ public void visit(IOperatorDescriptor op) {
+ op.contributeActivities(builder);
+ }
+ });
+ builder.finish();
+ final JobActivityGraph jag = builder.getActivityGraph();
+
+ JobRun run = new JobRun(jobId, jag);
+
+ run.setStatus(JobStatus.INITIALIZED, null);
+
+ ccs.getActiveRunMap().put(jobId, run);
+ final Set<Constraint> contributedConstraints = new HashSet<Constraint>();
+ final IConstraintAcceptor acceptor = new IConstraintAcceptor() {
+ @Override
+ public void addConstraint(Constraint constraint) {
+ contributedConstraints.add(constraint);
+ }
+ };
+ PlanUtils.visit(spec, new IOperatorDescriptorVisitor() {
+ @Override
+ public void visit(IOperatorDescriptor op) {
+ op.contributeSchedulingConstraints(acceptor, jag, appCtx);
+ }
+ });
+ PlanUtils.visit(spec, new IConnectorDescriptorVisitor() {
+ @Override
+ public void visit(IConnectorDescriptor conn) {
+ conn.contributeSchedulingConstraints(acceptor, jag, appCtx);
+ }
+ });
+ contributedConstraints.addAll(spec.getUserConstraints());
+
+ JobScheduler jrs = new JobScheduler(ccs, run, contributedConstraints);
+ run.setScheduler(jrs);
+ appCtx.notifyJobCreation(jobId, spec);
run.setStatus(JobStatus.RUNNING, null);
try {
run.getScheduler().startJob();
} catch (Exception e) {
ccs.getWorkQueue().schedule(new JobCleanupWork(ccs, run.getJobId(), JobStatus.FAILURE, e));
}
- callback.setValue(null);
+ callback.setValue(jobId);
} catch (Exception e) {
callback.setException(e);
}
diff --git a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
index 80ab5ec..dea4a8d 100644
--- a/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
+++ b/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobletCleanupNotificationWork.java
@@ -83,7 +83,6 @@
JSONObject jobLogObject = new JSONObject();
try {
JobActivityGraph jag = run.getJobActivityGraph();
- jobLogObject.put("job-specification", jag.getJobSpecification().toJSON());
jobLogObject.put("job-activity-graph", jag.toJSON());
jobLogObject.put("job-run", run.toJSON());
} catch (JSONException e) {
diff --git a/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java b/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
index ec7cf89..10c9c3d 100644
--- a/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
+++ b/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Joblet.java
@@ -97,7 +97,7 @@
deallocatableRegistry = new DefaultDeallocatableRegistry();
fileFactory = new WorkspaceFileFactory(this, (IOManager) appCtx.getRootContext().getIOManager());
cleanupPending = false;
- IJobletEventListenerFactory jelf = jag.getJobSpecification().getJobletEventListenerFactory();
+ IJobletEventListenerFactory jelf = jag.getJobletEventListenerFactory();
if (jelf != null) {
IJobletEventListener listener = jelf.createListener(this);
this.jobletEventListener = listener;
@@ -105,7 +105,7 @@
} else {
jobletEventListener = null;
}
- IGlobalJobDataFactory gjdf = jag.getJobSpecification().getGlobalJobDataFactory();
+ IGlobalJobDataFactory gjdf = jag.getGlobalJobDataFactory();
globalJobData = gjdf != null ? gjdf.createGlobalJobData(this) : null;
}
diff --git a/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java b/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java
index 78101bf..8d1b2a6 100644
--- a/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java
+++ b/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/StartTasksWork.java
@@ -26,11 +26,11 @@
import edu.uci.ics.hyracks.api.comm.IPartitionCollector;
import edu.uci.ics.hyracks.api.comm.IPartitionWriterFactory;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.ActivityId;
import edu.uci.ics.hyracks.api.dataflow.ConnectorDescriptorId;
import edu.uci.ics.hyracks.api.dataflow.IActivity;
import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
import edu.uci.ics.hyracks.api.dataflow.TaskId;
import edu.uci.ics.hyracks.api.dataflow.connectors.IConnectorPolicy;
@@ -90,20 +90,22 @@
IRecordDescriptorProvider rdp = new IRecordDescriptorProvider() {
@Override
- public RecordDescriptor getOutputRecordDescriptor(OperatorDescriptorId opId, int outputIndex) {
- return jag.getJobSpecification().getOperatorOutputRecordDescriptor(opId, outputIndex);
+ public RecordDescriptor getOutputRecordDescriptor(ActivityId aid, int outputIndex) {
+ IConnectorDescriptor conn = jag.getActivityOutputMap().get(aid).get(outputIndex);
+ return jag.getConnectorRecordDescriptorMap().get(conn.getConnectorId());
}
@Override
- public RecordDescriptor getInputRecordDescriptor(OperatorDescriptorId opId, int inputIndex) {
- return jag.getJobSpecification().getOperatorInputRecordDescriptor(opId, inputIndex);
+ public RecordDescriptor getInputRecordDescriptor(ActivityId aid, int inputIndex) {
+ IConnectorDescriptor conn = jag.getActivityInputMap().get(aid).get(inputIndex);
+ return jag.getConnectorRecordDescriptorMap().get(conn.getConnectorId());
}
};
for (TaskAttemptDescriptor td : taskDescriptors) {
TaskAttemptId taId = td.getTaskAttemptId();
TaskId tid = taId.getTaskId();
- IActivity han = jag.getActivityNodeMap().get(tid.getActivityId());
+ IActivity han = jag.getActivityMap().get(tid.getActivityId());
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("Initializing " + taId + " -> " + han);
}
@@ -113,7 +115,7 @@
List<IPartitionCollector> collectors = new ArrayList<IPartitionCollector>();
- List<IConnectorDescriptor> inputs = jag.getActivityInputConnectorDescriptors(tid.getActivityId());
+ List<IConnectorDescriptor> inputs = jag.getActivityInputMap().get(tid.getActivityId());
if (inputs != null) {
for (int i = 0; i < inputs.size(); ++i) {
IConnectorDescriptor conn = inputs.get(i);
@@ -121,17 +123,17 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info("input: " + i + ": " + conn.getConnectorId());
}
- RecordDescriptor recordDesc = jag.getJobSpecification().getConnectorRecordDescriptor(conn);
+ RecordDescriptor recordDesc = jag.getConnectorRecordDescriptorMap().get(conn.getConnectorId());
IPartitionCollector collector = createPartitionCollector(td, partition, task, i, conn,
recordDesc, cPolicy);
collectors.add(collector);
}
}
- List<IConnectorDescriptor> outputs = jag.getActivityOutputConnectorDescriptors(tid.getActivityId());
+ List<IConnectorDescriptor> outputs = jag.getActivityOutputMap().get(tid.getActivityId());
if (outputs != null) {
for (int i = 0; i < outputs.size(); ++i) {
final IConnectorDescriptor conn = outputs.get(i);
- RecordDescriptor recordDesc = jag.getJobSpecification().getConnectorRecordDescriptor(conn);
+ RecordDescriptor recordDesc = jag.getConnectorRecordDescriptorMap().get(conn.getConnectorId());
IConnectorPolicy cPolicy = connectorPoliciesMap.get(conn.getConnectorId());
IPartitionWriterFactory pwFactory = createPartitionWriterFactory(task, cPolicy, jobId, conn,
diff --git a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
index c4156f4..34f3e9d 100644
--- a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
+++ b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
@@ -230,9 +230,7 @@
data[1] = value;
writer.writeData(data);
}
- };
- ;
- ;
+ };;;
OutputCommitter outputCommitter = new org.apache.hadoop.mapreduce.lib.output.NullOutputFormat()
.getOutputCommitter(new TaskAttemptContext(conf, new TaskAttemptID()));
@@ -254,9 +252,7 @@
public Counter getCounter(Enum<?> arg0) {
return null;
}
- };
- ;
- ;
+ };;;
context = new org.apache.hadoop.mapreduce.Mapper().new Context(conf, new TaskAttemptID(),
newReader, recordWriter, outputCommitter, statusReporter,
(org.apache.hadoop.mapreduce.InputSplit) inputSplit);
@@ -308,8 +304,8 @@
inputSplitsProxy = new InputSplitsProxy(jobConf, splits);
}
- public HadoopMapperOperatorDescriptor(IOperatorDescriptorRegistry spec, JobConf jobConf, IHadoopClassFactory hadoopClassFactory)
- throws IOException {
+ public HadoopMapperOperatorDescriptor(IOperatorDescriptorRegistry spec, JobConf jobConf,
+ IHadoopClassFactory hadoopClassFactory) throws IOException {
super(spec, 1, getRecordDescriptor(jobConf, hadoopClassFactory), jobConf, hadoopClassFactory);
}
@@ -412,7 +408,7 @@
return createSelfReadingMapper(ctx, recordDescriptor, partition);
} else {
return new DeserializedOperatorNodePushable(ctx, new MapperOperator(partition),
- recordDescProvider.getInputRecordDescriptor(this.odId, 0));
+ recordDescProvider.getInputRecordDescriptor(this.activityNodeId, 0));
}
} catch (Exception e) {
throw new HyracksDataException(e);
diff --git a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReducerOperatorDescriptor.java b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReducerOperatorDescriptor.java
index 35df8de..86a4c3c 100644
--- a/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReducerOperatorDescriptor.java
+++ b/hyracks-dataflow-hadoop/src/main/java/edu/uci/ics/hyracks/dataflow/hadoop/HadoopReducerOperatorDescriptor.java
@@ -311,8 +311,8 @@
private IComparatorFactory comparatorFactory;
private boolean useAsCombiner = false;
- public HadoopReducerOperatorDescriptor(IOperatorDescriptorRegistry spec, JobConf conf, IComparatorFactory comparatorFactory,
- IHadoopClassFactory classFactory, boolean useAsCombiner) {
+ public HadoopReducerOperatorDescriptor(IOperatorDescriptorRegistry spec, JobConf conf,
+ IComparatorFactory comparatorFactory, IHadoopClassFactory classFactory, boolean useAsCombiner) {
super(spec, 1, getRecordDescriptor(conf, classFactory), conf, classFactory);
this.comparatorFactory = comparatorFactory;
this.useAsCombiner = useAsCombiner;
@@ -371,7 +371,7 @@
IOpenableDataWriterOperator op = new DeserializedPreclusteredGroupOperator(new int[] { 0 },
new IComparator[] { comparatorFactory.createComparator() }, new ReducerAggregator(createReducer()));
return new DeserializedOperatorNodePushable(ctx, op, recordDescProvider.getInputRecordDescriptor(
- getOperatorId(), 0));
+ getActivityId(), 0));
} catch (Exception e) {
throw new RuntimeException(e);
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java
index dd9e6ad..9048670 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/base/AbstractSingleActivityOperatorDescriptor.java
@@ -36,7 +36,7 @@
@Override
public final void contributeActivities(IActivityGraphBuilder builder) {
- builder.addActivity(this);
+ builder.addActivity(this, this);
for (int i = 0; i < getInputArity(); ++i) {
builder.addSourceEdge(i, this, i);
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
index 1d6b69d..a69fd01 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/connectors/OneToOneConnectorDescriptor.java
@@ -24,12 +24,11 @@
import edu.uci.ics.hyracks.api.constraints.IConstraintAcceptor;
import edu.uci.ics.hyracks.api.constraints.expressions.PartitionCountExpression;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.OperatorDescriptorId;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.job.IConnectorDescriptorRegistry;
import edu.uci.ics.hyracks.api.job.JobActivityGraph;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractConnectorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicChannelReader;
import edu.uci.ics.hyracks.dataflow.std.collectors.NonDeterministicFrameReader;
@@ -63,12 +62,11 @@
@Override
public void contributeSchedulingConstraints(IConstraintAcceptor constraintAcceptor, JobActivityGraph plan,
ICCApplicationContext appCtx) {
- JobSpecification jobSpec = plan.getJobSpecification();
- IOperatorDescriptor consumer = jobSpec.getConsumer(this);
- IOperatorDescriptor producer = jobSpec.getProducer(this);
+ OperatorDescriptorId consumer = plan.getConsumerActivity(getConnectorId()).getOperatorDescriptorId();
+ OperatorDescriptorId producer = plan.getProducerActivity(getConnectorId()).getOperatorDescriptorId();
- constraintAcceptor.addConstraint(new Constraint(new PartitionCountExpression(consumer.getOperatorId()),
- new PartitionCountExpression(producer.getOperatorId())));
+ constraintAcceptor.addConstraint(new Constraint(new PartitionCountExpression(consumer),
+ new PartitionCountExpression(producer)));
}
@Override
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
index 584d398..e05d80d 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/AbstractFileWriteOperatorDescriptor.java
@@ -91,6 +91,6 @@
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new DeserializedOperatorNodePushable(ctx, new FileWriteOperator(partition),
- recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
+ recordDescProvider.getInputRecordDescriptor(getActivityId(), 0));
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
index 2a205cc..6fb0189 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/file/PlainFileWriterOperatorDescriptor.java
@@ -50,7 +50,8 @@
* @param inputArity
* @param outputArity
*/
- public PlainFileWriterOperatorDescriptor(IOperatorDescriptorRegistry spec, IFileSplitProvider fileSplitProvider, String delim) {
+ public PlainFileWriterOperatorDescriptor(IOperatorDescriptorRegistry spec, IFileSplitProvider fileSplitProvider,
+ String delim) {
super(spec, 1, 0);
this.fileSplitProvider = fileSplitProvider;
this.delim = delim;
@@ -74,9 +75,9 @@
final FileSplit[] splits = fileSplitProvider.getFileSplits();
// Frame accessor
final FrameTupleAccessor frameTupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),
- recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
+ recordDescProvider.getInputRecordDescriptor(getActivityId(), 0));
// Record descriptor
- final RecordDescriptor recordDescriptor = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
+ final RecordDescriptor recordDescriptor = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
return new AbstractUnaryInputSinkOperatorNodePushable() {
private BufferedWriter out;
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java
index 9e64ebb..9fd45e9 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/external/ExternalGroupOperatorDescriptor.java
@@ -91,10 +91,10 @@
AggregateActivity aggregateAct = new AggregateActivity(new ActivityId(getOperatorId(), AGGREGATE_ACTIVITY_ID));
MergeActivity mergeAct = new MergeActivity(new ActivityId(odId, MERGE_ACTIVITY_ID));
- builder.addActivity(aggregateAct);
+ builder.addActivity(this, aggregateAct);
builder.addSourceEdge(0, aggregateAct, 0);
- builder.addActivity(mergeAct);
+ builder.addActivity(this, mergeAct);
builder.addTargetEdge(0, mergeAct, 0);
builder.addBlockingEdge(aggregateAct, mergeAct);
@@ -113,7 +113,7 @@
throws HyracksDataException {
return new ExternalGroupBuildOperatorNodePushable(ctx, new TaskId(getActivityId(), partition), keyFields,
framesLimit, comparatorFactories, firstNormalizerFactory, aggregatorFactory,
- recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0), recordDescriptors[0],
+ recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), recordDescriptors[0],
spillableTableFactory);
}
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupOperatorDescriptor.java
index 91f80a7..034e13f 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/hash/HashGroupOperatorDescriptor.java
@@ -47,9 +47,9 @@
private final int tableSize;
- public HashGroupOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys, ITuplePartitionComputerFactory tpcf,
- IBinaryComparatorFactory[] comparatorFactories, IAggregatorDescriptorFactory aggregatorFactory,
- RecordDescriptor outRecordDescriptor, int tableSize) {
+ public HashGroupOperatorDescriptor(IOperatorDescriptorRegistry spec, int[] keys,
+ ITuplePartitionComputerFactory tpcf, IBinaryComparatorFactory[] comparatorFactories,
+ IAggregatorDescriptorFactory aggregatorFactory, RecordDescriptor outRecordDescriptor, int tableSize) {
super(spec, 1, 1);
this.keys = keys;
this.tpcf = tpcf;
@@ -69,10 +69,10 @@
@Override
public void contributeActivities(IActivityGraphBuilder builder) {
HashBuildActivity ha = new HashBuildActivity(new ActivityId(odId, HASH_BUILD_ACTIVITY_ID));
- builder.addActivity(ha);
+ builder.addActivity(this, ha);
OutputActivity oa = new OutputActivity(new ActivityId(odId, OUTPUT_ACTIVITY_ID));
- builder.addActivity(oa);
+ builder.addActivity(this, oa);
builder.addSourceEdge(0, ha, 0);
builder.addTargetEdge(0, oa, 0);
@@ -91,7 +91,7 @@
final IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
return new HashGroupBuildOperatorNodePushable(ctx, new TaskId(getActivityId(), partition), keys, tpcf,
comparatorFactories, aggregatorFactory, tableSize, recordDescProvider.getInputRecordDescriptor(
- getOperatorId(), 0), recordDescriptors[0]);
+ getActivityId(), 0), recordDescriptors[0]);
}
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorDescriptor.java
index 9df5e44..d37b818 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/group/preclustered/PreclusteredGroupOperatorDescriptor.java
@@ -14,7 +14,6 @@
*/
package edu.uci.ics.hyracks.dataflow.std.group.preclustered;
-
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -47,6 +46,6 @@
final IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
throws HyracksDataException {
return new PreclusteredGroupOperatorNodePushable(ctx, groupFields, comparatorFactories, aggregatorFactory,
- recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0), recordDescriptors[0]);
+ recordDescProvider.getInputRecordDescriptor(getActivityId(), 0), recordDescriptors[0]);
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
index ef5cb94..6c58d6f 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/GraceHashJoinOperatorDescriptor.java
@@ -45,9 +45,10 @@
private final boolean isLeftOuter;
private final INullWriterFactory[] nullWriterFactories1;
- public GraceHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0, int recordsPerFrame,
- double factor, int[] keys0, int[] keys1, IBinaryHashFunctionFactory[] hashFunctionFactories,
- IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor) {
+ public GraceHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
+ int recordsPerFrame, double factor, int[] keys0, int[] keys1,
+ IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
+ RecordDescriptor recordDescriptor) {
super(spec, 2, 1);
this.memsize = memsize;
this.inputsize0 = inputsize0;
@@ -62,10 +63,10 @@
recordDescriptors[0] = recordDescriptor;
}
- public GraceHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0, int recordsPerFrame,
- double factor, int[] keys0, int[] keys1, IBinaryHashFunctionFactory[] hashFunctionFactories,
- IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor, boolean isLeftOuter,
- INullWriterFactory[] nullWriterFactories1) {
+ public GraceHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
+ int recordsPerFrame, double factor, int[] keys0, int[] keys1,
+ IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
+ RecordDescriptor recordDescriptor, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1) {
super(spec, 2, 1);
this.memsize = memsize;
this.inputsize0 = inputsize0;
@@ -82,19 +83,19 @@
@Override
public void contributeActivities(IActivityGraphBuilder builder) {
- HashPartitionActivityNode rpart = new HashPartitionActivityNode(new ActivityId(odId, RPARTITION_ACTIVITY_ID),
- keys0, 0);
- HashPartitionActivityNode spart = new HashPartitionActivityNode(new ActivityId(odId, SPARTITION_ACTIVITY_ID),
- keys1, 1);
- JoinActivityNode join = new JoinActivityNode(new ActivityId(odId, JOIN_ACTIVITY_ID));
+ ActivityId rpartAid = new ActivityId(odId, RPARTITION_ACTIVITY_ID);
+ HashPartitionActivityNode rpart = new HashPartitionActivityNode(rpartAid, keys0);
+ ActivityId spartAid = new ActivityId(odId, SPARTITION_ACTIVITY_ID);
+ HashPartitionActivityNode spart = new HashPartitionActivityNode(spartAid, keys1);
+ JoinActivityNode join = new JoinActivityNode(new ActivityId(odId, JOIN_ACTIVITY_ID), rpartAid, spartAid);
- builder.addActivity(rpart);
+ builder.addActivity(this, rpart);
builder.addSourceEdge(0, rpart, 0);
- builder.addActivity(spart);
+ builder.addActivity(this, spart);
builder.addSourceEdge(1, spart, 0);
- builder.addActivity(join);
+ builder.addActivity(this, join);
builder.addBlockingEdge(rpart, spart);
builder.addBlockingEdge(spart, join);
@@ -107,13 +108,11 @@
private class HashPartitionActivityNode extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
- private int operatorInputIndex;
private int keys[];
- public HashPartitionActivityNode(ActivityId id, int keys[], int operatorInputIndex) {
+ public HashPartitionActivityNode(ActivityId id, int keys[]) {
super(id);
this.keys = keys;
- this.operatorInputIndex = operatorInputIndex;
}
@Override
@@ -121,23 +120,28 @@
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new GraceHashJoinPartitionBuildOperatorNodePushable(ctx, new TaskId(getActivityId(), partition),
keys, hashFunctionFactories, comparatorFactories, (int) Math.ceil(Math.sqrt(inputsize0 * factor
- / nPartitions)), recordDescProvider.getInputRecordDescriptor(getOperatorId(),
- operatorInputIndex));
+ / nPartitions)), recordDescProvider.getInputRecordDescriptor(getActivityId(), 0));
}
}
private class JoinActivityNode extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
- public JoinActivityNode(ActivityId id) {
+ private final ActivityId rpartAid;
+
+ private final ActivityId spartAid;
+
+ public JoinActivityNode(ActivityId id, ActivityId rpartAid, ActivityId spartAid) {
super(id);
+ this.rpartAid = rpartAid;
+ this.spartAid = spartAid;
}
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
final IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
- final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
- final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
+ final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(rpartAid, 0);
+ final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(spartAid, 0);
int numPartitions = (int) Math.ceil(Math.sqrt(inputsize0 * factor / nPartitions));
return new GraceHashJoinOperatorNodePushable(ctx, new TaskId(new ActivityId(getOperatorId(),
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
index 1eaf3bfd..4f9b987 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
@@ -84,10 +84,10 @@
* @param recordDescriptor
* @throws HyracksDataException
*/
- public HybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0, int recordsPerFrame,
- double factor, int[] keys0, int[] keys1, IBinaryHashFunctionFactory[] hashFunctionFactories,
- IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor)
- throws HyracksDataException {
+ public HybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
+ int recordsPerFrame, double factor, int[] keys0, int[] keys1,
+ IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
+ RecordDescriptor recordDescriptor) throws HyracksDataException {
super(spec, 2, 1);
this.memsize = memsize;
this.inputsize0 = inputsize0;
@@ -102,10 +102,11 @@
recordDescriptors[0] = recordDescriptor;
}
- public HybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0, int recordsPerFrame,
- double factor, int[] keys0, int[] keys1, IBinaryHashFunctionFactory[] hashFunctionFactories,
- IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor, boolean isLeftOuter,
- INullWriterFactory[] nullWriterFactories1) throws HyracksDataException {
+ public HybridHashJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memsize, int inputsize0,
+ int recordsPerFrame, double factor, int[] keys0, int[] keys1,
+ IBinaryHashFunctionFactory[] hashFunctionFactories, IBinaryComparatorFactory[] comparatorFactories,
+ RecordDescriptor recordDescriptor, boolean isLeftOuter, INullWriterFactory[] nullWriterFactories1)
+ throws HyracksDataException {
super(spec, 2, 1);
this.memsize = memsize;
this.inputsize0 = inputsize0;
@@ -122,15 +123,15 @@
@Override
public void contributeActivities(IActivityGraphBuilder builder) {
- BuildAndPartitionActivityNode phase1 = new BuildAndPartitionActivityNode(new ActivityId(odId,
- BUILD_AND_PARTITION_ACTIVITY_ID));
- PartitionAndJoinActivityNode phase2 = new PartitionAndJoinActivityNode(new ActivityId(odId,
- PARTITION_AND_JOIN_ACTIVITY_ID));
+ ActivityId p1Aid = new ActivityId(odId, BUILD_AND_PARTITION_ACTIVITY_ID);
+ ActivityId p2Aid = new ActivityId(odId, PARTITION_AND_JOIN_ACTIVITY_ID);
+ BuildAndPartitionActivityNode phase1 = new BuildAndPartitionActivityNode(p1Aid, p2Aid);
+ PartitionAndJoinActivityNode phase2 = new PartitionAndJoinActivityNode(p2Aid, p1Aid);
- builder.addActivity(phase1);
+ builder.addActivity(this, phase1);
builder.addSourceEdge(1, phase1, 0);
- builder.addActivity(phase2);
+ builder.addActivity(this, phase2);
builder.addSourceEdge(0, phase2, 0);
builder.addBlockingEdge(phase1, phase2);
@@ -166,15 +167,18 @@
private class BuildAndPartitionActivityNode extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
- public BuildAndPartitionActivityNode(ActivityId id) {
+ private final ActivityId joinAid;
+
+ public BuildAndPartitionActivityNode(ActivityId id, ActivityId joinAid) {
super(id);
+ this.joinAid = joinAid;
}
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
- final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
- final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
+ final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(joinAid, 0);
+ final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
@@ -351,15 +355,18 @@
private class PartitionAndJoinActivityNode extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
- public PartitionAndJoinActivityNode(ActivityId id) {
+ private final ActivityId buildAid;
+
+ public PartitionAndJoinActivityNode(ActivityId id, ActivityId buildAid) {
super(id);
+ this.buildAid = buildAid;
}
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
- final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
- final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
+ final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+ final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(buildAid, 0);
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
index ddbe417..e0a5613 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
@@ -48,9 +48,6 @@
import edu.uci.ics.hyracks.dataflow.std.structures.SerializableHashTable;
public class InMemoryHashJoinOperatorDescriptor extends AbstractOperatorDescriptor {
- private static final int BUILD_ACTIVITY_ID = 0;
- private static final int PROBE_ACTIVITY_ID = 1;
-
private static final long serialVersionUID = 1L;
private final int[] keys0;
private final int[] keys1;
@@ -91,13 +88,15 @@
@Override
public void contributeActivities(IActivityGraphBuilder builder) {
- HashBuildActivityNode hba = new HashBuildActivityNode(new ActivityId(odId, 0));
- HashProbeActivityNode hpa = new HashProbeActivityNode(new ActivityId(odId, 1));
+ ActivityId hbaId = new ActivityId(odId, 0);
+ ActivityId hpaId = new ActivityId(odId, 1);
+ HashBuildActivityNode hba = new HashBuildActivityNode(hbaId, hpaId);
+ HashProbeActivityNode hpa = new HashProbeActivityNode(hpaId);
- builder.addActivity(hba);
+ builder.addActivity(this, hba);
builder.addSourceEdge(1, hba, 0);
- builder.addActivity(hpa);
+ builder.addActivity(this, hpa);
builder.addSourceEdge(0, hpa, 0);
builder.addTargetEdge(0, hpa, 0);
@@ -129,17 +128,18 @@
private class HashBuildActivityNode extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
- public HashBuildActivityNode(ActivityId id) {
+ private final ActivityId hpaId;
+
+ public HashBuildActivityNode(ActivityId id, ActivityId hpaId) {
super(id);
+ this.hpaId = hpaId;
}
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
- final RecordDescriptor rd0 = recordDescProvider
- .getInputRecordDescriptor(getOperatorId(), BUILD_ACTIVITY_ID);
- final RecordDescriptor rd1 = recordDescProvider
- .getInputRecordDescriptor(getOperatorId(), PROBE_ACTIVITY_ID);
+ final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(hpaId, 0);
+ final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
@@ -204,8 +204,8 @@
@Override
public void open() throws HyracksDataException {
- state = (HashBuildTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(),
- BUILD_ACTIVITY_ID), partition));
+ state = (HashBuildTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(), 0),
+ partition));
writer.open();
}
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
index 8d4f57a..a699703 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/NestedLoopJoinOperatorDescriptor.java
@@ -58,14 +58,15 @@
@Override
public void contributeActivities(IActivityGraphBuilder builder) {
- JoinCacheActivityNode jc = new JoinCacheActivityNode(new ActivityId(getOperatorId(), JOIN_CACHE_ACTIVITY_ID));
- NestedLoopJoinActivityNode nlj = new NestedLoopJoinActivityNode(new ActivityId(getOperatorId(),
- NL_JOIN_ACTIVITY_ID));
+ ActivityId jcaId = new ActivityId(getOperatorId(), JOIN_CACHE_ACTIVITY_ID);
+ ActivityId nljAid = new ActivityId(getOperatorId(), NL_JOIN_ACTIVITY_ID);
+ JoinCacheActivityNode jc = new JoinCacheActivityNode(jcaId, nljAid);
+ NestedLoopJoinActivityNode nlj = new NestedLoopJoinActivityNode(nljAid);
- builder.addActivity(jc);
+ builder.addActivity(this, jc);
builder.addSourceEdge(1, jc, 0);
- builder.addActivity(nlj);
+ builder.addActivity(this, nlj);
builder.addSourceEdge(0, nlj, 0);
builder.addTargetEdge(0, nlj, 0);
@@ -96,15 +97,18 @@
private class JoinCacheActivityNode extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
- public JoinCacheActivityNode(ActivityId id) {
+ private final ActivityId nljAid;
+
+ public JoinCacheActivityNode(ActivityId id, ActivityId nljAid) {
super(id);
+ this.nljAid = nljAid;
}
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) {
- final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
- final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
+ final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(nljAid, 0);
+ final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
final ITuplePairComparator comparator = comparatorFactory.createTuplePairComparator(ctx);
IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
index da02f3d..3a7ee2c 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -93,7 +93,6 @@
*/
public class OptimizedHybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor {
-
private static final int BUILD_AND_PARTITION_ACTIVITY_ID = 0;
private static final int PARTITION_AND_JOIN_ACTIVITY_ID = 1;
@@ -162,15 +161,15 @@
@Override
public void contributeActivities(IActivityGraphBuilder builder) {
- PartitionAndBuildActivityNode phase1 = new PartitionAndBuildActivityNode(new ActivityId(odId,
- BUILD_AND_PARTITION_ACTIVITY_ID));
- ProbeAndJoinActivityNode phase2 = new ProbeAndJoinActivityNode(new ActivityId(odId,
- PARTITION_AND_JOIN_ACTIVITY_ID));
+ ActivityId buildAid = new ActivityId(odId, BUILD_AND_PARTITION_ACTIVITY_ID);
+ ActivityId probeAid = new ActivityId(odId, PARTITION_AND_JOIN_ACTIVITY_ID);
+ PartitionAndBuildActivityNode phase1 = new PartitionAndBuildActivityNode(buildAid, probeAid);
+ ProbeAndJoinActivityNode phase2 = new ProbeAndJoinActivityNode(probeAid, buildAid);
- builder.addActivity(phase1);
+ builder.addActivity(this, phase1);
builder.addSourceEdge(0, phase1, 0);
- builder.addActivity(phase2);
+ builder.addActivity(this, phase2);
builder.addSourceEdge(1, phase2, 0);
builder.addBlockingEdge(phase1, phase2);
@@ -236,16 +235,19 @@
private class PartitionAndBuildActivityNode extends AbstractActivityNode {
private static final long serialVersionUID = 1L;
- public PartitionAndBuildActivityNode(ActivityId id) {
+ private final ActivityId probeAid;
+
+ public PartitionAndBuildActivityNode(ActivityId id, ActivityId probeAid) {
super(id);
+ this.probeAid = probeAid;
}
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
- final RecordDescriptor probeRd = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
- final RecordDescriptor buildRd = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
+ final RecordDescriptor probeRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+ final RecordDescriptor buildRd = recordDescProvider.getInputRecordDescriptor(probeAid, 0);
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
for (int i = 0; i < comparatorFactories.length; i++) {
@@ -314,16 +316,19 @@
private static final long serialVersionUID = 1L;
- public ProbeAndJoinActivityNode(ActivityId id) {
+ private final ActivityId buildAid;
+
+ public ProbeAndJoinActivityNode(ActivityId id, ActivityId buildAid) {
super(id);
+ this.buildAid = buildAid;
}
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
- final RecordDescriptor probeRd = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
- final RecordDescriptor buildRd = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 1);
+ final RecordDescriptor probeRd = recordDescProvider.getInputRecordDescriptor(buildAid, 0);
+ final RecordDescriptor buildRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
final ITuplePairComparator nljComparator0 = tuplePairComparatorFactory0.createTuplePairComparator(ctx);
final ITuplePairComparator nljComparator1 = tuplePairComparatorFactory1.createTuplePairComparator(ctx);
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java
index 7de8f95..30aae7f 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/map/DeserializedMapperOperatorDescriptor.java
@@ -64,8 +64,8 @@
private final IDeserializedMapperFactory mapperFactory;
- public DeserializedMapperOperatorDescriptor(IOperatorDescriptorRegistry spec, IDeserializedMapperFactory mapperFactory,
- RecordDescriptor recordDescriptor) {
+ public DeserializedMapperOperatorDescriptor(IOperatorDescriptorRegistry spec,
+ IDeserializedMapperFactory mapperFactory, RecordDescriptor recordDescriptor) {
super(spec, 1, 1);
this.mapperFactory = mapperFactory;
recordDescriptors[0] = recordDescriptor;
@@ -75,6 +75,6 @@
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new DeserializedOperatorNodePushable(ctx, new MapperOperator(),
- recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
+ recordDescProvider.getInputRecordDescriptor(getActivityId(), 0));
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
index d38b0a4..ba8c00c 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/MaterializingOperatorDescriptor.java
@@ -54,10 +54,10 @@
MaterializerActivityNode ma = new MaterializerActivityNode(new ActivityId(odId, MATERIALIZER_ACTIVITY_ID));
ReaderActivityNode ra = new ReaderActivityNode(new ActivityId(odId, READER_ACTIVITY_ID));
- builder.addActivity(ma);
+ builder.addActivity(this, ma);
builder.addSourceEdge(0, ma, 0);
- builder.addActivity(ra);
+ builder.addActivity(this, ra);
builder.addTargetEdge(0, ra, 0);
builder.addBlockingEdge(ma, ra);
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java
index 56a4f79..47a8616 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/PrinterOperatorDescriptor.java
@@ -64,6 +64,6 @@
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
return new DeserializedOperatorNodePushable(ctx, new PrinterOperator(),
- recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0));
+ recordDescProvider.getInputRecordDescriptor(getActivityId(), 0));
}
}
\ No newline at end of file
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java
index c8c8fb8..aad0e66 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/misc/SplitVectorOperatorDescriptor.java
@@ -105,7 +105,7 @@
}
};
return new DeserializedOperatorNodePushable(ctx, op, recordDescProvider.getInputRecordDescriptor(
- getOperatorId(), 0));
+ getActivityId(), 0));
}
}
@@ -158,8 +158,8 @@
writer.fail();
}
};
- return new DeserializedOperatorNodePushable(ctx, op, recordDescProvider.getInputRecordDescriptor(
- getOperatorId(), 0));
+ return new DeserializedOperatorNodePushable(ctx, op, recordDescProvider.getOutputRecordDescriptor(
+ getActivityId(), 0));
}
}
@@ -178,10 +178,10 @@
CollectActivity ca = new CollectActivity(new ActivityId(odId, COLLECT_ACTIVITY_ID));
SplitActivity sa = new SplitActivity(new ActivityId(odId, SPLIT_ACTIVITY_ID));
- builder.addActivity(ca);
+ builder.addActivity(this, ca);
builder.addSourceEdge(0, ca, 0);
- builder.addActivity(sa);
+ builder.addActivity(this, sa);
builder.addTargetEdge(0, sa, 0);
builder.addBlockingEdge(ca, sa);
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
index 708992b..be71b44 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/ExternalSortOperatorDescriptor.java
@@ -75,10 +75,10 @@
SortActivity sa = new SortActivity(new ActivityId(odId, SORT_ACTIVITY_ID));
MergeActivity ma = new MergeActivity(new ActivityId(odId, MERGE_ACTIVITY_ID));
- builder.addActivity(sa);
+ builder.addActivity(this, sa);
builder.addSourceEdge(0, sa, 0);
- builder.addActivity(ma);
+ builder.addActivity(this, ma);
builder.addTargetEdge(0, ma, 0);
builder.addBlockingEdge(sa, ma);
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
index be1e067..3b3c28d 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/InMemorySortOperatorDescriptor.java
@@ -67,10 +67,10 @@
SortActivity sa = new SortActivity(new ActivityId(odId, SORT_ACTIVITY_ID));
MergeActivity ma = new MergeActivity(new ActivityId(odId, MERGE_ACTIVITY_ID));
- builder.addActivity(sa);
+ builder.addActivity(this, sa);
builder.addSourceEdge(0, sa, 0);
- builder.addActivity(ma);
+ builder.addActivity(this, ma);
builder.addTargetEdge(0, ma, 0);
builder.addBlockingEdge(sa, ma);
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
index 23169b5..d9c16d8 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/sort/OptimizedExternalSortOperatorDescriptor.java
@@ -98,10 +98,10 @@
OptimizedSortActivity osa = new OptimizedSortActivity(new ActivityId(odId, SORT_ACTIVITY_ID));
OptimizedMergeActivity oma = new OptimizedMergeActivity(new ActivityId(odId, MERGE_ACTIVITY_ID));
- builder.addActivity(osa);
+ builder.addActivity(this, osa);
builder.addSourceEdge(0, osa, 0);
- builder.addActivity(oma);
+ builder.addActivity(this, oma);
builder.addTargetEdge(0, oma, 0);
builder.addBlockingEdge(osa, oma);
@@ -191,8 +191,8 @@
IOperatorNodePushable op = new AbstractUnaryOutputSourceOperatorNodePushable() {
@Override
public void initialize() throws HyracksDataException {
- OptimizedSortTaskState state = (OptimizedSortTaskState) ctx.getStateObject(new TaskId(new ActivityId(
- getOperatorId(), SORT_ACTIVITY_ID), partition));
+ OptimizedSortTaskState state = (OptimizedSortTaskState) ctx.getStateObject(new TaskId(
+ new ActivityId(getOperatorId(), SORT_ACTIVITY_ID), partition));
List<IFrameReader> runs = state.runs;
diff --git a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
index 6c93b61..7ac9f58 100644
--- a/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
+++ b/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
@@ -40,7 +40,7 @@
@Override
public void contributeActivities(IActivityGraphBuilder builder) {
UnionActivityNode uba = new UnionActivityNode(new ActivityId(getOperatorId(), 0));
- builder.addActivity(uba);
+ builder.addActivity(this, uba);
for (int i = 0; i < inputArity; ++i) {
builder.addSourceEdge(i, uba, i);
}
@@ -58,7 +58,7 @@
public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
throws HyracksDataException {
- RecordDescriptor inRecordDesc = recordDescProvider.getInputRecordDescriptor(getOperatorId(), 0);
+ RecordDescriptor inRecordDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
return new UnionOperator(ctx, inRecordDesc);
}
}
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
index 876b1a9..b6e8c72 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/InsertPipelineExample.java
@@ -88,8 +88,7 @@
JobSpecification job = createJob(options);
long start = System.currentTimeMillis();
- JobId jobId = hcc.createJob(options.app, job);
- hcc.start(jobId);
+ JobId jobId = hcc.startJob(options.app, job);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
System.err.println(start + " " + end + " " + (end - start));
@@ -153,8 +152,8 @@
// create operator descriptor
TreeIndexInsertUpdateDeleteOperatorDescriptor primaryInsert = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, recDesc, storageManager, indexRegistryProvider, primarySplitProvider, primaryTypeTraits,
- primaryComparatorFactories, primaryFieldPermutation, IndexOp.INSERT, dataflowHelperFactory,
- null, NoOpOperationCallbackProvider.INSTANCE);
+ primaryComparatorFactories, primaryFieldPermutation, IndexOp.INSERT, dataflowHelperFactory, null,
+ NoOpOperationCallbackProvider.INSTANCE);
JobHelper.createPartitionConstraint(spec, primaryInsert, splitNCs);
// prepare insertion into secondary index
@@ -177,8 +176,8 @@
// create operator descriptor
TreeIndexInsertUpdateDeleteOperatorDescriptor secondaryInsert = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, recDesc, storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
- secondaryComparatorFactories, secondaryFieldPermutation, IndexOp.INSERT, dataflowHelperFactory,
- null, NoOpOperationCallbackProvider.INSTANCE);
+ secondaryComparatorFactories, secondaryFieldPermutation, IndexOp.INSERT, dataflowHelperFactory, null,
+ NoOpOperationCallbackProvider.INSTANCE);
JobHelper.createPartitionConstraint(spec, secondaryInsert, splitNCs);
// end the insert pipeline at this sink operator
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
index 2c2d481..a6c7ea6 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexBulkLoadExample.java
@@ -86,8 +86,7 @@
JobSpecification job = createJob(options);
long start = System.currentTimeMillis();
- JobId jobId = hcc.createJob(options.app, job);
- hcc.start(jobId);
+ JobId jobId = hcc.startJob(options.app, job);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
System.err.println(start + " " + end + " " + (end - start));
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
index 9ce5d92..d24ba33 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/PrimaryIndexSearchExample.java
@@ -79,8 +79,7 @@
JobSpecification job = createJob(options);
long start = System.currentTimeMillis();
- JobId jobId = hcc.createJob(options.app, job);
- hcc.start(jobId);
+ JobId jobId = hcc.startJob(options.app, job);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
System.err.println(start + " " + end + " " + (end - start));
@@ -144,7 +143,7 @@
IIndexDataflowHelperFactory dataflowHelperFactory = new BTreeDataflowHelperFactory();
BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(spec, recDesc, storageManager,
indexRegistryProvider, btreeSplitProvider, typeTraits, comparatorFactories, lowKeyFields,
- highKeyFields, true, true, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ highKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
JobHelper.createPartitionConstraint(spec, btreeSearchOp, splitNCs);
// have each node print the results of its respective B-Tree
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
index 465af1a..5aa338a 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexBulkLoadExample.java
@@ -81,8 +81,7 @@
JobSpecification job = createJob(options);
long start = System.currentTimeMillis();
- JobId jobId = hcc.createJob(options.app, job);
- hcc.start(jobId);
+ JobId jobId = hcc.startJob(options.app, job);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
System.err.println(start + " " + end + " " + (end - start));
diff --git a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
index fd9e25d..277668b 100644
--- a/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
+++ b/hyracks-examples/btree-example/btreeclient/src/main/java/edu/uci/ics/hyracks/examples/btree/client/SecondaryIndexSearchExample.java
@@ -82,8 +82,7 @@
JobSpecification job = createJob(options);
long start = System.currentTimeMillis();
- JobId jobId = hcc.createJob(options.app, job);
- hcc.start(jobId);
+ JobId jobId = hcc.startJob(options.app, job);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
System.err.println(start + " " + end + " " + (end - start));
@@ -171,7 +170,7 @@
BTreeSearchOperatorDescriptor secondarySearchOp = new BTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
searchComparatorFactories, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
JobHelper.createPartitionConstraint(spec, secondarySearchOp, splitNCs);
// secondary index will output tuples with [UTF8String, Integer]
@@ -187,7 +186,7 @@
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, indexRegistryProvider, primarySplitProvider, primaryTypeTraits,
primaryComparatorFactories, primaryLowKeyFields, primaryHighKeyFields, true, true,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
JobHelper.createPartitionConstraint(spec, primarySearchOp, splitNCs);
// have each node print the results of its respective B-Tree
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
index b4159e2..ff29b6e 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/AbstractBTreeOperatorTest.java
@@ -184,9 +184,9 @@
public void createSecondaryIndex() throws Exception {
JobSpecification spec = new JobSpecification();
- TreeIndexCreateOperatorDescriptor secondaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
- indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits, secondaryComparatorFactories,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ TreeIndexCreateOperatorDescriptor secondaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
+ storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
+ secondaryComparatorFactories, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryCreateOp, NC1_ID);
spec.addRoot(secondaryCreateOp);
runTest(spec);
@@ -217,7 +217,7 @@
// scan primary index
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, indexRegistryProvider, primarySplitProvider, primaryTypeTraits,
- primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory,
+ primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
@@ -269,16 +269,16 @@
int[] primaryFieldPermutation = { 0, 1, 2, 4, 5, 7 };
TreeIndexInsertUpdateDeleteOperatorDescriptor primaryBtreeInsertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, ordersDesc, storageManager, indexRegistryProvider, primarySplitProvider, primaryTypeTraits,
- primaryComparatorFactories, primaryFieldPermutation, pipelineOperation, dataflowHelperFactory,
- null, NoOpOperationCallbackProvider.INSTANCE);
+ primaryComparatorFactories, primaryFieldPermutation, pipelineOperation, dataflowHelperFactory, null,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeInsertOp, NC1_ID);
// first secondary index
int[] fieldPermutationB = { 4, 0 };
TreeIndexInsertUpdateDeleteOperatorDescriptor secondaryInsertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, ordersDesc, storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
- secondaryComparatorFactories, fieldPermutationB, pipelineOperation, dataflowHelperFactory,
- null, NoOpOperationCallbackProvider.INSTANCE);
+ secondaryComparatorFactories, fieldPermutationB, pipelineOperation, dataflowHelperFactory, null,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryInsertOp, NC1_ID);
NullSinkOperatorDescriptor nullSink = new NullSinkOperatorDescriptor(spec);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
index 36b1364..a7c4847 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexScanOperatorTest.java
@@ -71,7 +71,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, indexRegistryProvider, primarySplitProvider, primaryTypeTraits,
- primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory,
+ primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
index 661e057..d92b269 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreePrimaryIndexSearchOperatorTest.java
@@ -76,7 +76,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, indexRegistryProvider, primarySplitProvider, primaryTypeTraits,
- primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory,
+ primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
index 4d0eefa..56e1c53 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexInsertOperatorTest.java
@@ -42,9 +42,9 @@
@Before
public void setup() throws Exception {
super.setup();
- createPrimaryIndex();
+ createPrimaryIndex();
loadPrimaryIndex();
- createSecondaryIndex();
+ createSecondaryIndex();
loadSecondaryIndex();
insertPipeline(false);
}
@@ -81,7 +81,7 @@
BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
secondaryRecDesc, storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
secondaryComparatorFactories, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
// second field from the tuples coming from secondary index
@@ -93,7 +93,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, indexRegistryProvider, primarySplitProvider, primaryTypeTraits,
primaryComparatorFactories, primaryLowKeyFields, primaryHighKeyFields, true, true,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
index 43fb138..dd56c88 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexSearchOperatorTest.java
@@ -42,9 +42,9 @@
@Before
public void setup() throws Exception {
super.setup();
- createPrimaryIndex();
+ createPrimaryIndex();
loadPrimaryIndex();
- createSecondaryIndex();
+ createSecondaryIndex();
loadSecondaryIndex();
}
@@ -80,7 +80,7 @@
BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
secondaryRecDesc, storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
secondaryComparatorFactories, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
int[] primaryLowKeyFields = { 1 }; // second field from the tuples
@@ -92,7 +92,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, indexRegistryProvider, primarySplitProvider, primaryTypeTraits,
primaryComparatorFactories, primaryLowKeyFields, primaryHighKeyFields, true, true,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
index d0dffae..5ba22ae 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/btree/BTreeSecondaryIndexUpsertOperatorTest.java
@@ -27,9 +27,9 @@
@Before
public void setup() throws Exception {
super.setup();
- createPrimaryIndex();
+ createPrimaryIndex();
loadPrimaryIndex();
- createSecondaryIndex();
+ createSecondaryIndex();
loadSecondaryIndex();
insertPipeline(true);
}
@@ -66,7 +66,7 @@
BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
secondaryRecDesc, storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
secondaryComparatorFactories, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
// second field from the tuples coming from secondary index
@@ -78,7 +78,7 @@
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, indexRegistryProvider, primarySplitProvider, primaryTypeTraits,
primaryComparatorFactories, primaryLowKeyFields, primaryHighKeyFields, true, true,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/WordInvertedIndexTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/WordInvertedIndexTest.java
index 35cf8ed..e250b1f 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/WordInvertedIndexTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/invertedindex/WordInvertedIndexTest.java
@@ -218,7 +218,7 @@
int[] highKeyFields = null; // + infinity
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, indexRegistryProvider, primaryFileSplitProvider, primaryTypeTraits,
- primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, btreeDataflowHelperFactory,
+ primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, btreeDataflowHelperFactory, false,
NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
return primaryBtreeSearchOp;
@@ -314,7 +314,7 @@
InvertedIndexSearchOperatorDescriptor invIndexSearchOp = new InvertedIndexSearchOperatorDescriptor(spec, 0,
storageManager, btreeFileSplitProvider, invListsFileSplitProvider, indexRegistryProvider,
tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits, invListsComparatorFactories,
- btreeDataflowHelperFactory, tokenizerFactory, searchModifierFactory, invListsRecDesc,
+ btreeDataflowHelperFactory, tokenizerFactory, searchModifierFactory, invListsRecDesc, false,
NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, invIndexSearchOp, NC1_ID);
return invIndexSearchOp;
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
index 5afc94f..b57f3f4 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/AbstractRTreeOperatorTest.java
@@ -264,7 +264,7 @@
// scan primary index
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, indexRegistryProvider, primarySplitProvider, primaryTypeTraits,
- primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, btreeDataflowHelperFactory,
+ primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, btreeDataflowHelperFactory, false,
NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
@@ -313,8 +313,8 @@
int[] primaryFieldPermutation = { 0, 1, 2, 4, 5, 7, 9, 10, 11, 12 };
TreeIndexInsertUpdateDeleteOperatorDescriptor primaryInsertOp = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
spec, ordersDesc, storageManager, indexRegistryProvider, primarySplitProvider, primaryTypeTraits,
- primaryComparatorFactories, primaryFieldPermutation, IndexOp.INSERT, btreeDataflowHelperFactory,
- null, NoOpOperationCallbackProvider.INSTANCE);
+ primaryComparatorFactories, primaryFieldPermutation, IndexOp.INSERT, btreeDataflowHelperFactory, null,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryInsertOp, NC1_ID);
// secondary index
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
index a965e5f..36e96e3 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexInsertOperatorTest.java
@@ -85,7 +85,7 @@
RTreeSearchOperatorDescriptor secondarySearchOp = new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
- secondaryComparatorFactories, keyFields, rtreeDataflowHelperFactory,
+ secondaryComparatorFactories, keyFields, rtreeDataflowHelperFactory, false,
NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
@@ -98,7 +98,7 @@
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, indexRegistryProvider, primarySplitProvider, primaryTypeTraits,
primaryComparatorFactories, primaryLowKeyFields, primaryHighKeyFields, true, true,
- btreeDataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ btreeDataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
index 654b0b9..c2aeead 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexScanOperatorTest.java
@@ -82,7 +82,7 @@
RTreeSearchOperatorDescriptor secondarySearchOp = new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
- secondaryComparatorFactories, keyFields, rtreeDataflowHelperFactory,
+ secondaryComparatorFactories, keyFields, rtreeDataflowHelperFactory, false,
NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
index 5a8cf59..00d08dd 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/am/rtree/RTreeSecondaryIndexSearchOperatorTest.java
@@ -84,7 +84,7 @@
RTreeSearchOperatorDescriptor secondarySearchOp = new RTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
- secondaryComparatorFactories, keyFields, rtreeDataflowHelperFactory,
+ secondaryComparatorFactories, keyFields, rtreeDataflowHelperFactory, false,
NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp, NC1_ID);
@@ -97,7 +97,7 @@
BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
storageManager, indexRegistryProvider, primarySplitProvider, primaryTypeTraits,
primaryComparatorFactories, primaryLowKeyFields, primaryHighKeyFields, true, true,
- btreeDataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ btreeDataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primarySearchOp, NC1_ID);
IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
index 4ed8361..023bdd9 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -108,11 +108,10 @@
}
protected void runTest(JobSpecification spec) throws Exception {
- JobId jobId = hcc.createJob("test", spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(spec.toJSON().toString(2));
}
- hcc.start(jobId);
+ JobId jobId = hcc.startJob("test", spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(jobId.toString());
}
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
index dec42c1..e0b8c73 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
@@ -105,11 +105,10 @@
}
protected void runTest(JobSpecification spec) throws Exception {
- JobId jobId = hcc.createJob("test", spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(spec.toJSON().toString(2));
}
- hcc.start(jobId);
+ JobId jobId = hcc.startJob("test", spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(jobId.toString());
}
diff --git a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java
index d556285..0d5a627 100644
--- a/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java
+++ b/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java
@@ -87,8 +87,7 @@
/**
* Test of aggregations using locality aware connector. The output two files should be the
- * same, each of which is the aggregation of two copies of the lineitem.tbl.
- *
+ * same, each of which is the aggregation of two copies of the lineitem.tbl.
* Note that if the hashing connector is not working correctly, the two files may be different. This
* also means that even the output files are the same, the hashing may have other problems.
*
@@ -99,66 +98,47 @@
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, "asterix-001", "asterix-002", "asterix-003", "asterix-004");
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, "asterix-001", "asterix-002",
+ "asterix-003", "asterix-004");
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
int tableSize = 8;
- HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
- spec,
- keyFields,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }),
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new IntSumFieldAggregatorFactory(3, true),
- new FloatSumFieldAggregatorFactory(5, true) }),
- outputRec, tableSize);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true),
+ new FloatSumFieldAggregatorFactory(5, true) }), outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- "asterix-005", "asterix-006");
-
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, "asterix-005", "asterix-006");
+
BitSet nodemap = new BitSet(8);
-
+
nodemap.set(0);
nodemap.set(2);
nodemap.set(5);
nodemap.set(7);
- IConnectorDescriptor conn1 = new LocalityAwareMToNPartitioningConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new LocalityAwareMToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }), new HashtableLocalityMap(nodemap));
-
- new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "localityAwareSumInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "localityAwareSumInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- "asterix-005", "asterix-006");
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, "asterix-005", "asterix-006");
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -166,11 +146,11 @@
spec.addRoot(printer);
runTest(spec);
}
-
+
/**
* Test for locality aware connector, using the global hashing node mapper. This should have
* the exactly the same result as using {@link MToNPartitioningConnectorDescriptor}.
- *
+ *
* @throws Exception
*/
@Test
@@ -178,59 +158,40 @@
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, "asterix-001", "asterix-002", "asterix-003", "asterix-004");
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, "asterix-001", "asterix-002",
+ "asterix-003", "asterix-004");
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
int tableSize = 8;
- HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
- spec,
- keyFields,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }),
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new IntSumFieldAggregatorFactory(3, true),
- new FloatSumFieldAggregatorFactory(5, true) }),
- outputRec, tableSize);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true),
+ new FloatSumFieldAggregatorFactory(5, true) }), outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- "asterix-005", "asterix-006");
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, "asterix-005", "asterix-006");
- IConnectorDescriptor conn1 = new LocalityAwareMToNPartitioningConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new LocalityAwareMToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }), new GlobalHashingLocalityMap());
-
- new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "localityAwareSumInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "localityAwareSumInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- "asterix-005", "asterix-006");
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, "asterix-005", "asterix-006");
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -238,16 +199,14 @@
spec.addRoot(printer);
runTest(spec);
}
-
- private AbstractSingleActivityOperatorDescriptor getPrinter(
- IOperatorDescriptorRegistry spec, String prefix) throws IOException {
- AbstractSingleActivityOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(
- spec, new ConstantFileSplitProvider(new FileSplit[] {
- new FileSplit("asterix-005", createTempFile()
- .getAbsolutePath()),
- new FileSplit("asterix-006", createTempFile()
- .getAbsolutePath()) }), "\t");
+ private AbstractSingleActivityOperatorDescriptor getPrinter(IOperatorDescriptorRegistry spec, String prefix)
+ throws IOException {
+
+ AbstractSingleActivityOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec,
+ new ConstantFileSplitProvider(new FileSplit[] {
+ new FileSplit("asterix-005", createTempFile().getAbsolutePath()),
+ new FileSplit("asterix-006", createTempFile().getAbsolutePath()) }), "\t");
return printer;
}
diff --git a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
index cb226c2..943f232 100644
--- a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
+++ b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/ExternalGroupClient.java
@@ -122,8 +122,7 @@
System.out.print(i + "\t" + (System.currentTimeMillis() - start));
start = System.currentTimeMillis();
- JobId jobId = hcc.createJob(options.app, job);
- hcc.start(jobId);
+ JobId jobId = hcc.startJob(options.app, job);
hcc.waitForCompletion(jobId);
System.out.println("\t" + (System.currentTimeMillis() - start));
}
@@ -201,8 +200,8 @@
switch (alg) {
case 0: // new external hash graph
- grouper = new edu.uci.ics.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor(spec, keys,
- framesLimit, new IBinaryComparatorFactory[] {
+ grouper = new edu.uci.ics.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor(spec,
+ keys, framesLimit, new IBinaryComparatorFactory[] {
// PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
new IntegerNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
@@ -238,8 +237,8 @@
PointableBinaryHashFunctionFactory.of(IntegerPointable.FACTORY) }));
spec.connect(scanSortConn2, fileScanner, 0, sorter2, 0);
- grouper = new edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor(spec, keys,
- new IBinaryComparatorFactory[] {
+ grouper = new edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor(
+ spec, keys, new IBinaryComparatorFactory[] {
// PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
new MultiFieldsAggregatorFactory(
@@ -274,8 +273,8 @@
spec.connect(scanConn2, fileScanner, 0, grouper, 0);
break;
default:
- grouper = new edu.uci.ics.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor(spec, keys,
- framesLimit, new IBinaryComparatorFactory[] {
+ grouper = new edu.uci.ics.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor(spec,
+ keys, framesLimit, new IBinaryComparatorFactory[] {
// PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY),
PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
new IntegerNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
diff --git a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
index 0e06274..31019ab 100644
--- a/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
+++ b/hyracks-examples/text-example/textclient/src/main/java/edu/uci/ics/hyracks/examples/text/client/WordCountMain.java
@@ -101,9 +101,8 @@
options.algo, options.htSize, options.sbSize, options.format);
long start = System.currentTimeMillis();
- JobId jobId = hcc.createJob(options.app, job, options.runtimeProfiling ? EnumSet.of(JobFlag.PROFILE_RUNTIME)
+ JobId jobId = hcc.startJob(options.app, job, options.runtimeProfiling ? EnumSet.of(JobFlag.PROFILE_RUNTIME)
: EnumSet.noneOf(JobFlag.class));
- hcc.start(jobId);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
System.err.println(start + " " + end + " " + (end - start));
diff --git a/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java b/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
index eda17c8..01ccdef 100644
--- a/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
+++ b/hyracks-examples/tpch-example/tpchclient/src/main/java/edu/uci/ics/hyracks/examples/tpch/client/Main.java
@@ -127,9 +127,8 @@
options.graceFactor, options.memSize, options.tableSize, options.hasGroupBy);
long start = System.currentTimeMillis();
- JobId jobId = hcc.createJob(options.app, job,
+ JobId jobId = hcc.startJob(options.app, job,
options.profile ? EnumSet.of(JobFlag.PROFILE_RUNTIME) : EnumSet.noneOf(JobFlag.class));
- hcc.start(jobId);
hcc.waitForCompletion(jobId);
long end = System.currentTimeMillis();
System.err.println(start + " " + end + " " + (end - start));
diff --git a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
index eadf679..dfd229a 100644
--- a/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
+++ b/hyracks-hadoop-compat/src/main/java/edu/uci/ics/hyracks/hadoop/compat/client/HyracksClient.java
@@ -66,11 +66,10 @@
JobId jobId;
if (doProfiling) {
System.out.println("PROFILING");
- jobId = connection.createJob(applicationName, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
+ jobId = connection.startJob(applicationName, spec, EnumSet.of(JobFlag.PROFILE_RUNTIME));
} else {
- jobId = connection.createJob(applicationName, spec);
+ jobId = connection.startJob(applicationName, spec);
}
- connection.start(jobId);
HyracksRunningJob runningJob = new HyracksRunningJob(jobId, spec, this);
return runningJob;
}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
index 4a1921e..f403312 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorDescriptor.java
@@ -46,9 +46,9 @@
IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories, int[] lowKeyFields, int[] highKeyFields,
boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory dataflowHelperFactory,
- IOperationCallbackProvider opCallbackProvider) {
+ boolean retainInput, IOperationCallbackProvider opCallbackProvider) {
super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
- comparatorFactories, dataflowHelperFactory, null, opCallbackProvider);
+ comparatorFactories, dataflowHelperFactory, null, retainInput, opCallbackProvider);
this.lowKeyFields = lowKeyFields;
this.highKeyFields = highKeyFields;
this.lowKeyInclusive = lowKeyInclusive;
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
index 9dda175..b284847 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeSearchOperatorNodePushable.java
@@ -38,7 +38,6 @@
super(opDesc, ctx, partition, recordDescProvider);
this.lowKeyInclusive = lowKeyInclusive;
this.highKeyInclusive = highKeyInclusive;
- this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
if (lowKeyFields != null && lowKeyFields.length > 0) {
lowKey = new PermutingFrameTupleReference();
lowKey.setFieldPermutation(lowKeyFields);
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
index 1675986..1fc5cb8 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorDescriptor.java
@@ -41,10 +41,10 @@
IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories, int[] lowKeyFields, int[] highKeyFields,
boolean lowKeyInclusive, boolean highKeyInclusive, IIndexDataflowHelperFactory dataflowHelperFactory,
- IOperationCallbackProvider opCallbackProvider, ITupleUpdaterFactory tupleUpdaterFactory) {
+ boolean retainInput, IOperationCallbackProvider opCallbackProvider, ITupleUpdaterFactory tupleUpdaterFactory) {
super(spec, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits, comparatorFactories,
lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive, dataflowHelperFactory,
- opCallbackProvider);
+ retainInput, opCallbackProvider);
this.tupleUpdaterFactory = tupleUpdaterFactory;
}
diff --git a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
index e96b510..1141639 100644
--- a/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
+++ b/hyracks-storage-am-btree/src/main/java/edu/uci/ics/hyracks/storage/am/btree/dataflow/BTreeUpdateSearchOperatorNodePushable.java
@@ -42,10 +42,17 @@
}
@Override
- protected void writeSearchResults() throws Exception {
+ protected void writeSearchResults(int tupleIndex) throws Exception {
while (cursor.hasNext()) {
tb.reset();
cursor.next();
+ if (retainInput) {
+ frameTuple.reset(accessor, tupleIndex);
+ for (int i = 0; i < frameTuple.getFieldCount(); i++) {
+ dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
+ tb.addFieldEndOffset();
+ }
+ }
ITupleReference tuple = cursor.getTuple();
tupleUpdater.updateTuple(tuple);
for (int i = 0; i < tuple.getFieldCount(); i++) {
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
index a36b503..9f0fbc9 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/AbstractTreeIndexOperatorDescriptor.java
@@ -42,6 +42,7 @@
protected final IIndexDataflowHelperFactory dataflowHelperFactory;
protected final ITupleFilterFactory tupleFilterFactory;
+ protected final boolean retainInput;
protected final IOperationCallbackProvider opCallbackProvider;
public AbstractTreeIndexOperatorDescriptor(IOperatorDescriptorRegistry spec,
@@ -53,7 +54,7 @@
IBinaryComparatorFactory[] comparatorFactories,
IIndexDataflowHelperFactory dataflowHelperFactory,
ITupleFilterFactory tupleFilterFactory,
- IOperationCallbackProvider opCallbackProvider) {
+ boolean retainInput, IOperationCallbackProvider opCallbackProvider) {
super(spec, inputArity, outputArity);
this.fileSplitProvider = fileSplitProvider;
this.storageManager = storageManager;
@@ -61,6 +62,7 @@
this.typeTraits = typeTraits;
this.comparatorFactories = comparatorFactories;
this.dataflowHelperFactory = dataflowHelperFactory;
+ this.retainInput = retainInput;
this.tupleFilterFactory = tupleFilterFactory;
this.opCallbackProvider = opCallbackProvider;
if (outputArity > 0) {
@@ -104,6 +106,11 @@
}
@Override
+ public boolean getRetainInput() {
+ return retainInput;
+ }
+
+ @Override
public IOperationCallbackProvider getOpCallbackProvider() {
return opCallbackProvider;
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java
index 6872e3f..e37d374 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/IIndexOperatorDescriptor.java
@@ -32,5 +32,7 @@
public IIndexDataflowHelperFactory getIndexDataflowHelperFactory();
+ public boolean getRetainInput();
+
public IOperationCallbackProvider getOpCallbackProvider();
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
index 26a5275..0020089 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorDescriptor.java
@@ -38,7 +38,7 @@
float fillFactor, IIndexDataflowHelperFactory dataflowHelperFactory,
IOperationCallbackProvider opCallbackProvider) {
super(spec, 1, 0, null, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
- comparatorFactories, dataflowHelperFactory, null, opCallbackProvider);
+ comparatorFactories, dataflowHelperFactory, null, false, opCallbackProvider);
this.fieldPermutation = fieldPermutation;
this.fillFactor = fillFactor;
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
index e30ce85..bb0689a 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexBulkLoadOperatorNodePushable.java
@@ -49,7 +49,7 @@
public void open() throws HyracksDataException {
AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexHelper
.getOperatorDescriptor();
- RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
+ RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
accessor = new FrameTupleAccessor(treeIndexHelper.getHyracksTaskContext().getFrameSize(), recDesc);
try {
treeIndexHelper.init(false);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorDescriptor.java
index f5cf09e..075a6a4 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexCreateOperatorDescriptor.java
@@ -34,7 +34,7 @@
ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories,
IIndexDataflowHelperFactory dataflowHelperFactory, IOperationCallbackProvider opCallbackProvider) {
super(spec, 0, 0, null, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
- comparatorFactories, dataflowHelperFactory, null, opCallbackProvider);
+ comparatorFactories, dataflowHelperFactory, null, false, opCallbackProvider);
}
@Override
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
index bf6e403..324485e 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorDescriptor.java
@@ -34,7 +34,7 @@
IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
IIndexDataflowHelperFactory dataflowHelperFactory, IOperationCallbackProvider opCallbackProvider) {
super(spec, 0, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits, null,
- dataflowHelperFactory, null, opCallbackProvider);
+ dataflowHelperFactory, null, false, opCallbackProvider);
}
@Override
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
index 0ea59af..a615386 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
@@ -42,7 +42,7 @@
IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation, IndexOp op,
IIndexDataflowHelperFactory dataflowHelperFactory, ITupleFilterFactory tupleFilterFactory, IOperationCallbackProvider opCallbackProvider) {
super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
- comparatorFactories, dataflowHelperFactory, tupleFilterFactory, opCallbackProvider);
+ comparatorFactories, dataflowHelperFactory, tupleFilterFactory, false, opCallbackProvider);
this.fieldPermutation = fieldPermutation;
this.op = op;
}
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
index b839c72..017cb75 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
@@ -56,7 +56,7 @@
public void open() throws HyracksDataException {
AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexHelper
.getOperatorDescriptor();
- RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
+ RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
accessor = new FrameTupleAccessor(treeIndexHelper.getHyracksTaskContext().getFrameSize(), inputRecDesc);
writeBuffer = treeIndexHelper.getHyracksTaskContext().allocateFrame();
writer.open();
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexSearchOperatorNodePushable.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexSearchOperatorNodePushable.java
index fd44311..7e23d29 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexSearchOperatorNodePushable.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexSearchOperatorNodePushable.java
@@ -25,6 +25,7 @@
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
@@ -34,7 +35,7 @@
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
public abstract class TreeIndexSearchOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
- protected TreeIndexDataflowHelper treeIndexHelper;
+ protected final TreeIndexDataflowHelper treeIndexHelper;
protected FrameTupleAccessor accessor;
protected ByteBuffer writeBuffer;
@@ -48,13 +49,16 @@
protected ITreeIndexFrame cursorFrame;
protected IIndexAccessor indexAccessor;
- protected RecordDescriptor recDesc;
+ protected final RecordDescriptor inputRecDesc;
+ protected final boolean retainInput;
+ protected FrameTupleReference frameTuple;
public TreeIndexSearchOperatorNodePushable(AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx,
int partition, IRecordDescriptorProvider recordDescProvider) {
treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
opDesc, ctx, partition);
- this.recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
+ this.retainInput = treeIndexHelper.getOperatorDescriptor().getRetainInput();
+ inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
}
protected abstract ISearchPredicate createSearchPredicate();
@@ -67,38 +71,46 @@
@Override
public void open() throws HyracksDataException {
- accessor = new FrameTupleAccessor(treeIndexHelper.getHyracksTaskContext().getFrameSize(), recDesc);
+ accessor = new FrameTupleAccessor(treeIndexHelper.getHyracksTaskContext().getFrameSize(), inputRecDesc);
writer.open();
- try {
+ try {
treeIndexHelper.init(false);
treeIndex = (ITreeIndex) treeIndexHelper.getIndex();
cursorFrame = treeIndex.getLeafFrameFactory().createFrame();
searchPred = createSearchPredicate();
writeBuffer = treeIndexHelper.getHyracksTaskContext().allocateFrame();
- tb = new ArrayTupleBuilder(treeIndex.getFieldCount());
+ tb = new ArrayTupleBuilder(recordDesc.getFieldCount());
dos = tb.getDataOutput();
appender = new FrameTupleAppender(treeIndexHelper.getHyracksTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
indexAccessor = treeIndex.createAccessor(treeIndexHelper.getModificationOperationCallback(),
treeIndexHelper.getSearchOperationCallback());
cursor = createCursor();
+ if (retainInput) {
+ frameTuple = new FrameTupleReference();
+ }
} catch (Exception e) {
treeIndexHelper.deinit();
throw new HyracksDataException(e);
}
}
- protected void writeSearchResults() throws Exception {
+ protected void writeSearchResults(int tupleIndex) throws Exception {
while (cursor.hasNext()) {
tb.reset();
cursor.next();
-
+ if (retainInput) {
+ frameTuple.reset(accessor, tupleIndex);
+ for (int i = 0; i < frameTuple.getFieldCount(); i++) {
+ dos.write(frameTuple.getFieldData(i), frameTuple.getFieldStart(i), frameTuple.getFieldLength(i));
+ tb.addFieldEndOffset();
+ }
+ }
ITupleReference tuple = cursor.getTuple();
for (int i = 0; i < tuple.getFieldCount(); i++) {
dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
tb.addFieldEndOffset();
}
-
if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
FrameUtils.flushFrame(writeBuffer, writer);
appender.reset(writeBuffer, true);
@@ -118,7 +130,7 @@
resetSearchPredicate(i);
cursor.reset();
indexAccessor.search(cursor, searchPred);
- writeSearchResults();
+ writeSearchResults(i);
}
} catch (Exception e) {
throw new HyracksDataException(e);
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
index 9eae626..6bf0983 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/dataflow/TreeIndexStatsOperatorDescriptor.java
@@ -38,7 +38,7 @@
ITypeTraits[] typeTraits, IBinaryComparatorFactory[] comparatorFactories,
IIndexDataflowHelperFactory dataflowHelperFactory, IOperationCallbackProvider opCallbackProvider) {
super(spec, 0, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
- comparatorFactories, dataflowHelperFactory, null, opCallbackProvider);
+ comparatorFactories, dataflowHelperFactory, null, false, opCallbackProvider);
}
@Override
diff --git a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/MultiComparator.java b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/MultiComparator.java
index e7be912..a8ddc19 100644
--- a/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/MultiComparator.java
+++ b/hyracks-storage-am-common/src/main/java/edu/uci/ics/hyracks/storage/am/common/ophelpers/MultiComparator.java
@@ -51,6 +51,22 @@
}
return 0;
}
+
+ public int compare(ITupleReference tupleA,
+ ITupleReference tupleB, int startFieldIndex) {
+ for (int i = 0; i < cmps.length; i++) {
+ int ix = startFieldIndex + i;
+ int cmp = cmps[i].compare(tupleA.getFieldData(ix),
+ tupleA.getFieldStart(ix), tupleA.getFieldLength(ix),
+ tupleB.getFieldData(ix), tupleB.getFieldStart(ix),
+ tupleB.getFieldLength(ix));
+ if (cmp < 0)
+ return -1;
+ else if (cmp > 0)
+ return 1;
+ }
+ return 0;
+ }
public int fieldRangeCompare(ITupleReference tupleA, ITupleReference tupleB, int startFieldIndex, int numFields) {
for (int i = startFieldIndex; i < startFieldIndex + numFields; i++) {
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearchModifier.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearchModifier.java
index 8f7a3e3..bd96f67 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearchModifier.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/api/IInvertedIndexSearchModifier.java
@@ -18,7 +18,7 @@
import java.util.List;
public interface IInvertedIndexSearchModifier {
- int getOccurrenceThreshold(List<IInvertedListCursor> invListCursors);
+ public int getOccurrenceThreshold(List<IInvertedListCursor> invListCursors);
- int getPrefixLists(List<IInvertedListCursor> invListCursors);
+ public int getPrefixLists(List<IInvertedListCursor> invListCursors);
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
index 3b21fec..117f96f 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/AbstractInvertedIndexOperatorDescriptor.java
@@ -44,6 +44,7 @@
// General.
protected final IStorageManagerInterface storageManager;
protected final IIndexRegistryProvider<IIndex> indexRegistryProvider;
+ protected final boolean retainInput;
protected final IOperationCallbackProvider opCallbackProvider;
// Btree.
@@ -66,12 +67,13 @@
IIndexRegistryProvider<IIndex> indexRegistryProvider, ITypeTraits[] tokenTypeTraits,
IBinaryComparatorFactory[] tokenComparatorFactories, ITypeTraits[] invListsTypeTraits,
IBinaryComparatorFactory[] invListComparatorFactories, IBinaryTokenizerFactory tokenizerFactory,
- IIndexDataflowHelperFactory btreeDataflowHelperFactory, IOperationCallbackProvider opCallbackProvider) {
+ IIndexDataflowHelperFactory btreeDataflowHelperFactory, boolean retainInput, IOperationCallbackProvider opCallbackProvider) {
super(spec, inputArity, outputArity);
// General.
this.storageManager = storageManager;
this.indexRegistryProvider = indexRegistryProvider;
+ this.retainInput = retainInput;
this.opCallbackProvider = opCallbackProvider;
// Btree.
@@ -150,6 +152,11 @@
}
@Override
+ public boolean getRetainInput() {
+ return retainInput;
+ }
+
+ @Override
public IOperationCallbackProvider getOpCallbackProvider() {
return opCallbackProvider;
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
index 94409de..a146479 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/BinaryTokenizerOperatorDescriptor.java
@@ -46,9 +46,9 @@
}
@Override
- public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx, IRecordDescriptorProvider recordDescProvider,
- int partition, int nPartitions) throws HyracksDataException {
- return new BinaryTokenizerOperatorNodePushable(ctx, recordDescProvider.getInputRecordDescriptor(odId, 0),
- recordDescriptors[0], tokenizerFactory.createTokenizer(), tokenFields, keyFields);
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) throws HyracksDataException {
+ return new BinaryTokenizerOperatorNodePushable(ctx, recordDescProvider.getInputRecordDescriptor(
+ getActivityId(), 0), recordDescriptors[0], tokenizerFactory.createTokenizer(), tokenFields, keyFields);
}
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
index cc586ae..7ffec4d 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorDescriptor.java
@@ -44,7 +44,8 @@
IOperationCallbackProvider opCallbackProvider) {
super(spec, 1, 0, null, storageManager, btreeFileSplitProvider, invListsFileSplitProvider,
indexRegistryProvider, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
- invListComparatorFactories, tokenizerFactory, btreeDataflowHelperFactory, opCallbackProvider);
+ invListComparatorFactories, tokenizerFactory, btreeDataflowHelperFactory, false,
+ opCallbackProvider);
this.fieldPermutation = fieldPermutation;
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java
index 6da7481..62e6b31 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexBulkLoadOperatorNodePushable.java
@@ -52,7 +52,7 @@
public void open() throws HyracksDataException {
AbstractInvertedIndexOperatorDescriptor opDesc = (AbstractInvertedIndexOperatorDescriptor) btreeDataflowHelper
.getOperatorDescriptor();
- RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
+ RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
accessor = new FrameTupleAccessor(btreeDataflowHelper.getHyracksTaskContext().getFrameSize(), recDesc);
// BTree.
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexCreateOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexCreateOperatorDescriptor.java
index 540d794..01d1a97 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexCreateOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexCreateOperatorDescriptor.java
@@ -42,7 +42,7 @@
IOperationCallbackProvider opCallbackProvider) {
super(spec, 0, 0, null, storageManager, btreeFileSplitProvider, invListsFileSplitProvider,
indexRegistryProvider, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
- invListComparatorFactories, tokenizerFactory, btreeDataflowHelperFactory, opCallbackProvider);
+ invListComparatorFactories, tokenizerFactory, btreeDataflowHelperFactory, false, opCallbackProvider);
}
@Override
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexDataflowHelper.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexDataflowHelper.java
index 15d73b9..e769016 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexDataflowHelper.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexDataflowHelper.java
@@ -52,8 +52,7 @@
invIndexOpDesc.getInvListsTypeTraits());
return new InvertedIndex(opDesc.getStorageManager().getBufferCache(ctx), btree,
invIndexOpDesc.getInvListsTypeTraits(), invIndexOpDesc.getInvListsComparatorFactories(),
- invListBuilder, invIndexOpDesc.getTokenizerFactory().createTokenizer(), opDesc.getStorageManager()
- .getFileMapProvider(ctx),
- opDesc.getFileSplitProvider().getFileSplits()[partition].getLocalFile());
+ invListBuilder, opDesc.getStorageManager().getFileMapProvider(ctx), opDesc.getFileSplitProvider()
+ .getFileSplits()[partition].getLocalFile());
}
}
\ No newline at end of file
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java
index f5f92fc..497dfb5 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorDescriptor.java
@@ -45,11 +45,12 @@
ITypeTraits[] tokenTypeTraits, IBinaryComparatorFactory[] tokenComparatorFactories,
ITypeTraits[] invListsTypeTraits, IBinaryComparatorFactory[] invListComparatorFactories,
IIndexDataflowHelperFactory btreeDataflowHelperFactory, IBinaryTokenizerFactory queryTokenizerFactory,
- IInvertedIndexSearchModifierFactory searchModifierFactory, RecordDescriptor recDesc,
+ IInvertedIndexSearchModifierFactory searchModifierFactory, RecordDescriptor recDesc, boolean retainInput,
IOperationCallbackProvider opCallbackProvider) {
super(spec, 1, 1, recDesc, storageManager, btreeFileSplitProvider, invListsFileSplitProvider,
indexRegistryProvider, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
- invListComparatorFactories, queryTokenizerFactory, btreeDataflowHelperFactory, opCallbackProvider);
+ invListComparatorFactories, queryTokenizerFactory, btreeDataflowHelperFactory, retainInput,
+ opCallbackProvider);
this.queryField = queryField;
this.searchModifierFactory = searchModifierFactory;
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorNodePushable.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorNodePushable.java
index 4b506aa..4529823 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorNodePushable.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/dataflow/InvertedIndexSearchOperatorNodePushable.java
@@ -57,6 +57,7 @@
private DataOutput dos;
private final AbstractInvertedIndexOperatorDescriptor opDesc;
+ private final boolean retainInput;
public InvertedIndexSearchOperatorNodePushable(AbstractInvertedIndexOperatorDescriptor opDesc,
IHyracksTaskContext ctx, int partition, int queryField, IInvertedIndexSearchModifier searchModifier,
@@ -66,14 +67,15 @@
.createIndexDataflowHelper(opDesc, ctx, partition);
invIndexDataflowHelper = new InvertedIndexDataflowHelper(btreeDataflowHelper, opDesc, ctx, partition);
this.queryField = queryField;
- this.searchPred = new InvertedIndexSearchPredicate(searchModifier);
+ this.searchPred = new InvertedIndexSearchPredicate(opDesc.getTokenizerFactory().createTokenizer(), searchModifier);
this.recordDescProvider = recordDescProvider;
+ this.retainInput = invIndexDataflowHelper.getOperatorDescriptor().getRetainInput();
}
@Override
public void open() throws HyracksDataException {
- RecordDescriptor recDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getOperatorId(), 0);
- accessor = new FrameTupleAccessor(btreeDataflowHelper.getHyracksTaskContext().getFrameSize(), recDesc);
+ RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
+ accessor = new FrameTupleAccessor(btreeDataflowHelper.getHyracksTaskContext().getFrameSize(), inputRecDesc);
tuple = new FrameTupleReference();
// BTree.
try {
@@ -102,12 +104,13 @@
}
writeBuffer = btreeDataflowHelper.getHyracksTaskContext().allocateFrame();
- tb = new ArrayTupleBuilder(opDesc.getInvListsTypeTraits().length);
+ tb = new ArrayTupleBuilder(recordDesc.getFieldCount());
dos = tb.getDataOutput();
appender = new FrameTupleAppender(btreeDataflowHelper.getHyracksTaskContext().getFrameSize());
appender.reset(writeBuffer, true);
indexAccessor = invIndex.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+ //InvertedIndex.InvertedIndexAccessor accessor =
resultCursor = indexAccessor.createSearchCursor();
writer.open();
}
@@ -116,6 +119,12 @@
while (resultCursor.hasNext()) {
resultCursor.next();
tb.reset();
+ if (retainInput) {
+ for (int i = 0; i < tuple.getFieldCount(); i++) {
+ dos.write(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+ tb.addFieldEndOffset();
+ }
+ }
ITupleReference invListElement = resultCursor.getTuple();
int invListFields = opDesc.getInvListsTypeTraits().length;
for (int i = 0; i < invListFields; i++) {
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListBuilder.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListBuilder.java
index 643c105..c4ab601 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListBuilder.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListBuilder.java
@@ -36,7 +36,7 @@
@Override
public boolean startNewList(ITupleReference tuple, int tokenField) {
- if (pos + listElementSize > targetBuf.length)
+ if (pos + listElementSize >= targetBuf.length)
return false;
else {
listSize = 0;
@@ -46,7 +46,7 @@
@Override
public boolean appendElement(ITupleReference tuple, int numTokenFields, int numElementFields) {
- if (pos + listElementSize > targetBuf.length)
+ if (pos + listElementSize >= targetBuf.length)
return false;
for (int i = 0; i < numElementFields; i++) {
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListCursor.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListCursor.java
index 4158019..446f171 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListCursor.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/FixedSizeElementInvertedListCursor.java
@@ -57,7 +57,7 @@
@Override
public void next() {
- if (currentOff + elementSize >= bufferCache.getPageSize()) {
+ if (currentOff + 2 * elementSize >= bufferCache.getPageSize()) {
currentPageIx++;
currentOff = 0;
} else {
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
index b43a594..ada05e6 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndex.java
@@ -25,13 +25,11 @@
import edu.uci.ics.hyracks.api.io.IIOManager;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ByteArrayAccessibleOutputStream;
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.storage.am.btree.impls.BTree;
import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoadContext;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexBulkLoader;
import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -47,7 +45,6 @@
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearcher;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListBuilder;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
@@ -72,7 +69,6 @@
private final ITypeTraits[] invListTypeTraits;
private final IBinaryComparatorFactory[] invListCmpFactories;
private final IInvertedListBuilder invListBuilder;
- private final IBinaryTokenizer tokenizer;
private final int numTokenFields;
private final int numInvListKeys;
@@ -80,14 +76,13 @@
public InvertedIndex(IBufferCache bufferCache, BTree btree, ITypeTraits[] invListTypeTraits,
IBinaryComparatorFactory[] invListCmpFactories, IInvertedListBuilder invListBuilder,
- IBinaryTokenizer tokenizer, IFileMapProvider fileMapProvider, FileReference file) {
+ IFileMapProvider fileMapProvider, FileReference file) {
this.bufferCache = bufferCache;
this.fileMapProvider = fileMapProvider;
this.btree = btree;
this.invListTypeTraits = invListTypeTraits;
this.invListCmpFactories = invListCmpFactories;
this.invListBuilder = invListBuilder;
- this.tokenizer = tokenizer;
this.numTokenFields = btree.getComparatorFactories().length;
this.numInvListKeys = invListCmpFactories.length;
this.file = file;
@@ -236,26 +231,26 @@
public final class InvertedIndexBulkLoader implements IIndexBulkLoader {
private final ArrayTupleBuilder btreeTupleBuilder;
private final ArrayTupleReference btreeTupleReference;
- private final float btreeFillFactor;
- private IIndexBulkLoadContext btreeBulkLoadCtx;
private final IIndexBulkLoader btreeBulkloader;
private int currentInvListStartPageId;
private int currentInvListStartOffset;
- private final ByteArrayAccessibleOutputStream currentInvListTokenBaaos = new ByteArrayAccessibleOutputStream();
- private final FixedSizeTupleReference currentInvListToken = new FixedSizeTupleReference(invListTypeTraits);
+ private final ArrayTupleBuilder lastTupleBuilder;
+ private final ArrayTupleReference lastTuple;
private int currentPageId;
private ICachedPage currentPage;
private final MultiComparator tokenCmp;
+ private final MultiComparator invListCmp;
public InvertedIndexBulkLoader(float btreeFillFactor, int startPageId, int fileId) throws IndexException,
HyracksDataException {
this.tokenCmp = MultiComparator.create(btree.getComparatorFactories());
+ this.invListCmp = MultiComparator.create(invListCmpFactories);
this.btreeTupleBuilder = new ArrayTupleBuilder(btree.getFieldCount());
this.btreeTupleReference = new ArrayTupleReference();
- this.btreeFillFactor = btreeFillFactor;
-
+ this.lastTupleBuilder = new ArrayTupleBuilder(numTokenFields + numInvListKeys);
+ this.lastTuple = new ArrayTupleReference();
this.btreeBulkloader = btree.createBulkLoader(btreeFillFactor);
currentPageId = startPageId;
currentPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, currentPageId), true);
@@ -274,7 +269,8 @@
private void createAndInsertBTreeTuple() throws HyracksDataException {
// Build tuple.
btreeTupleBuilder.reset();
- btreeTupleBuilder.addField(currentInvListTokenBaaos.getByteArray(), 0, currentInvListTokenBaaos.size());
+ btreeTupleBuilder.addField(lastTuple.getFieldData(0), lastTuple.getFieldStart(0),
+ lastTuple.getFieldLength(0));
btreeTupleBuilder.addField(IntegerSerializerDeserializer.INSTANCE, currentInvListStartPageId);
btreeTupleBuilder.addField(IntegerSerializerDeserializer.INSTANCE, currentPageId);
btreeTupleBuilder.addField(IntegerSerializerDeserializer.INSTANCE, currentInvListStartOffset);
@@ -294,18 +290,18 @@
*/
@Override
public void add(ITupleReference tuple) throws HyracksDataException {
- // First inverted list, copy token to baaos and start new list.
- if (currentInvListTokenBaaos.size() == 0) {
- currentInvListStartPageId = currentPageId;
- currentInvListStartOffset = invListBuilder.getPos();
-
- // Remember current token.
- currentInvListTokenBaaos.reset();
- for (int i = 0; i < numTokenFields; i++) {
- currentInvListTokenBaaos.write(tuple.getFieldData(i), tuple.getFieldStart(i),
- tuple.getFieldLength(i));
+ boolean firstElement = lastTupleBuilder.getSize() == 0;
+ boolean startNewList = firstElement;
+ if (!firstElement) {
+ // If the current and the last token don't match, we start a new list.
+ lastTuple.reset(lastTupleBuilder.getFieldEndOffsets(), lastTupleBuilder.getByteArray());
+ startNewList = tokenCmp.compare(tuple, lastTuple) != 0;
+ }
+ if (startNewList) {
+ if (!firstElement) {
+ // Create entry in btree for last inverted list.
+ createAndInsertBTreeTuple();
}
-
if (!invListBuilder.startNewList(tuple, numTokenFields)) {
pinNextPage();
invListBuilder.setTargetBuffer(currentPage.getBuffer().array(), 0);
@@ -313,34 +309,13 @@
throw new IllegalStateException("Failed to create first inverted list.");
}
}
- }
-
- // Create new inverted list?
- currentInvListToken.reset(currentInvListTokenBaaos.getByteArray(), 0);
- if (tokenCmp.compare(tuple, currentInvListToken) != 0) {
-
- // Create entry in btree for last inverted list.
- createAndInsertBTreeTuple();
-
- // Remember new token.
- currentInvListTokenBaaos.reset();
- for (int i = 0; i < numTokenFields; i++) {
- currentInvListTokenBaaos.write(tuple.getFieldData(i), tuple.getFieldStart(i),
- tuple.getFieldLength(i));
- }
-
- // Start new list.
- if (!invListBuilder.startNewList(tuple, numTokenFields)) {
- pinNextPage();
- invListBuilder.setTargetBuffer(currentPage.getBuffer().array(), 0);
- if (!invListBuilder.startNewList(tuple, numTokenFields)) {
- throw new IllegalStateException(
- "Failed to start new inverted list after switching to a new page.");
- }
- }
-
currentInvListStartPageId = currentPageId;
currentInvListStartOffset = invListBuilder.getPos();
+ } else {
+ if (invListCmp.compare(tuple, lastTuple, numTokenFields) == 0) {
+ // Duplicate inverted-list element.
+ return;
+ }
}
// Append to current inverted list.
@@ -352,6 +327,13 @@
"Failed to append element to inverted list after switching to a new page.");
}
}
+
+ // Remember last tuple by creating a copy.
+ // TODO: This portion can be optimized by only copying the token when it changes, and using the last appended inverted-list element as a reference.
+ lastTupleBuilder.reset();
+ for (int i = 0; i < tuple.getFieldCount(); i++) {
+ lastTupleBuilder.addField(tuple.getFieldData(i), tuple.getFieldStart(i), tuple.getFieldLength(i));
+ }
}
@Override
@@ -391,7 +373,7 @@
private final IInvertedIndexSearcher searcher;
public InvertedIndexAccessor(InvertedIndex index) {
- this.searcher = new TOccurrenceSearcher(ctx, index, tokenizer);
+ this.searcher = new TOccurrenceSearcher(ctx, index);
}
@Override
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndexSearchPredicate.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndexSearchPredicate.java
index 989878d..9e9a2c3 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndexSearchPredicate.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/InvertedIndexSearchPredicate.java
@@ -19,15 +19,18 @@
import edu.uci.ics.hyracks.storage.am.common.api.ISearchPredicate;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
public class InvertedIndexSearchPredicate implements ISearchPredicate {
private static final long serialVersionUID = 1L;
private ITupleReference queryTuple;
private int queryFieldIndex;
- private final IInvertedIndexSearchModifier searchModifier;
+ private final IBinaryTokenizer queryTokenizer;
+ private final IInvertedIndexSearchModifier searchModifier;
- public InvertedIndexSearchPredicate(IInvertedIndexSearchModifier searchModifier) {
+ public InvertedIndexSearchPredicate(IBinaryTokenizer queryTokenizer, IInvertedIndexSearchModifier searchModifier) {
+ this.queryTokenizer = queryTokenizer;
this.searchModifier = searchModifier;
}
@@ -51,6 +54,10 @@
return searchModifier;
}
+ public IBinaryTokenizer getQueryTokenizer() {
+ return queryTokenizer;
+ }
+
@Override
public MultiComparator getLowKeyComparator() {
// TODO: This doesn't make sense for an inverted index. Change ISearchPredicate interface.
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
index bf4857e..c5f823a 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcher.java
@@ -50,6 +50,8 @@
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
+// TODO: The search procedure is rather confusing regarding cursor positions, hasNext() calls etc.
+// Needs an overhaul some time.
public class TOccurrenceSearcher implements IInvertedIndexSearcher {
protected final IHyracksCommonContext ctx;
@@ -80,7 +82,6 @@
protected final InvertedIndex invIndex;
protected final MultiComparator invListCmp;
- protected final IBinaryTokenizer queryTokenizer;
protected final ITypeTraits[] invListFieldsWithCount;
protected int occurrenceThreshold;
@@ -88,11 +89,10 @@
protected List<IInvertedListCursor> invListCursorCache = new ArrayList<IInvertedListCursor>(cursorCacheSize);
protected List<IInvertedListCursor> invListCursors = new ArrayList<IInvertedListCursor>(cursorCacheSize);
- public TOccurrenceSearcher(IHyracksCommonContext ctx, InvertedIndex invIndex, IBinaryTokenizer queryTokenizer) {
+ public TOccurrenceSearcher(IHyracksCommonContext ctx, InvertedIndex invIndex) {
this.ctx = ctx;
this.invIndex = invIndex;
this.invListCmp = MultiComparator.create(invIndex.getInvListElementCmpFactories());
- this.queryTokenizer = queryTokenizer;
leafFrame = invIndex.getBTree().getLeafFrameFactory().createFrame();
interiorFrame = invIndex.getBTree().getInteriorFrameFactory().createFrame();
@@ -150,6 +150,7 @@
ITupleReference queryTuple = searchPred.getQueryTuple();
int queryFieldIndex = searchPred.getQueryFieldIndex();
IInvertedIndexSearchModifier searchModifier = searchPred.getSearchModifier();
+ IBinaryTokenizer queryTokenizer = searchPred.getQueryTokenizer();
queryTokenAppender.reset(queryTokenFrame, true);
queryTokenizer.reset(queryTuple.getFieldData(queryFieldIndex), queryTuple.getFieldStart(queryFieldIndex),
@@ -349,7 +350,9 @@
if (advanceCursor) {
invListTidx++;
- invListCursor.next();
+ if (invListCursor.hasNext()) {
+ invListCursor.next();
+ }
}
}
@@ -440,7 +443,9 @@
if (advanceCursor) {
invListTidx++;
- invListCursor.next();
+ if (invListCursor.hasNext()) {
+ invListCursor.next();
+ }
}
}
@@ -449,7 +454,9 @@
ITupleReference invListTuple = invListCursor.getTuple();
newBufIdx = appendTupleToNewResults(invListTuple, 1, newBufIdx);
invListTidx++;
- invListCursor.next();
+ if (invListCursor.hasNext()) {
+ invListCursor.next();
+ }
}
// append remaining elements from previous result set
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixProbeOnly.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixProbeOnly.java
index b40b9cf..957ecc0 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixProbeOnly.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixProbeOnly.java
@@ -23,15 +23,13 @@
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
public class TOccurrenceSearcherSuffixProbeOnly extends TOccurrenceSearcher {
protected final MultiComparator invListCmp;
- public TOccurrenceSearcherSuffixProbeOnly(IHyracksTaskContext ctx, InvertedIndex invIndex,
- IBinaryTokenizer queryTokenizer) {
- super(ctx, invIndex, queryTokenizer);
+ public TOccurrenceSearcherSuffixProbeOnly(IHyracksTaskContext ctx, InvertedIndex invIndex) {
+ super(ctx, invIndex);
this.invListCmp = MultiComparator.create(invIndex.getInvListElementCmpFactories());
}
diff --git a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixScanOnly.java b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixScanOnly.java
index ab48ac7..bd9bd60 100644
--- a/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixScanOnly.java
+++ b/hyracks-storage-am-invertedindex/src/main/java/edu/uci/ics/hyracks/storage/am/invertedindex/impls/TOccurrenceSearcherSuffixScanOnly.java
@@ -24,15 +24,13 @@
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
public class TOccurrenceSearcherSuffixScanOnly extends TOccurrenceSearcher {
protected final MultiComparator invListCmp;
- public TOccurrenceSearcherSuffixScanOnly(IHyracksTaskContext ctx, InvertedIndex invIndex,
- IBinaryTokenizer queryTokenizer) {
- super(ctx, invIndex, queryTokenizer);
+ public TOccurrenceSearcherSuffixScanOnly(IHyracksTaskContext ctx, InvertedIndex invIndex) {
+ super(ctx, invIndex);
this.invListCmp = MultiComparator.create(invIndex.getInvListElementCmpFactories());
}
diff --git a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
index 1af3187..9818dce 100644
--- a/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
+++ b/hyracks-storage-am-rtree/src/main/java/edu/uci/ics/hyracks/storage/am/rtree/dataflow/RTreeSearchOperatorDescriptor.java
@@ -40,9 +40,9 @@
IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
IBinaryComparatorFactory[] comparatorFactories, int[] keyFields,
- IIndexDataflowHelperFactory dataflowHelperFactory, IOperationCallbackProvider opCallbackProvider) {
+ IIndexDataflowHelperFactory dataflowHelperFactory, boolean retainInput, IOperationCallbackProvider opCallbackProvider) {
super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
- comparatorFactories, dataflowHelperFactory, null, opCallbackProvider);
+ comparatorFactories, dataflowHelperFactory, null, false, opCallbackProvider);
this.keyFields = keyFields;
}
diff --git a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
index aa53a39..c652151 100644
--- a/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
+++ b/hyracks-storage-common/src/main/java/edu/uci/ics/hyracks/storage/common/buffercache/BufferCache.java
@@ -38,7 +38,7 @@
private static final int MAP_FACTOR = 2;
private static final int MAX_VICTIMIZATION_TRY_COUNT = 5;
- private static final int MAX_WAIT_FOR_CLEANER_THREAD_TIME = 1000;
+ private static final int MAX_WAIT_FOR_CLEANER_THREAD_TIME = 1000 * 60;
private static final int MIN_CLEANED_COUNT_DIFF = 4;
private final int maxOpenFiles;
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
index 0c2c2fe..5e40c05 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/AbstractInvIndexSearchTest.java
@@ -167,8 +167,8 @@
invListCmpFactories[0] = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY);
IInvertedListBuilder invListBuilder = new FixedSizeElementInvertedListBuilder(invListTypeTraits);
- invIndex = new InvertedIndex(bufferCache, btree, invListTypeTraits, invListCmpFactories, invListBuilder,
- tokenizer, fmp, invListsFile);
+ invIndex = new InvertedIndex(bufferCache, btree, invListTypeTraits, invListCmpFactories, invListBuilder, fmp,
+ invListsFile);
invIndex.create();
invIndex.open();
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
index 645a851..3c2d656 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/BulkLoadTest.java
@@ -131,7 +131,7 @@
IInvertedListBuilder invListBuilder = new FixedSizeElementInvertedListBuilder(invListTypeTraits);
InvertedIndex invIndex = new InvertedIndex(bufferCache, btree, invListTypeTraits, invListCmpFactories,
- invListBuilder, null, fmp, invListsFile);
+ invListBuilder, fmp, invListsFile);
invIndex.create();
invIndex.open();
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java
index b2642d9..ccb3cc3 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchPerfTest.java
@@ -48,7 +48,6 @@
* queries from a list of predefined tokens in the index, and measure the
* performance of executing them with different search modifiers. We test the
* ConjunctiveSearchModifier and the JaccardSearchModifier.
- *
*/
public class SearchPerfTest extends AbstractInvIndexSearchTest {
@@ -114,7 +113,6 @@
* Determine the expected results with the ScanCount algorithm. The
* ScanCount algorithm is very simple, so we can be confident the results
* are correct.
- *
*/
protected void fillExpectedResults(int[] queryTokenIndexes, int numQueryTokens, int occurrenceThreshold) {
// reset scan count array
@@ -145,15 +143,13 @@
* tokens. We run each query, measure it's time, and verify it's results
* against the results produced by ScanCount, implemented in
* fillExpectedResults().
- *
*/
private void runQueries(IInvertedIndexSearchModifier searchModifier, int numQueries) throws Exception {
-
rnd.setSeed(50);
InvertedIndexAccessor accessor = (InvertedIndexAccessor) invIndex.createAccessor(
NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- InvertedIndexSearchPredicate searchPred = new InvertedIndexSearchPredicate(searchModifier);
+ InvertedIndexSearchPredicate searchPred = new InvertedIndexSearchPredicate(tokenizer, searchModifier);
// generate random queries
int[] queryTokenIndexes = new int[tokens.size()];
@@ -240,7 +236,6 @@
/**
* Runs 50 random conjunctive search queries to test the
* ConjunctiveSearchModifier.
- *
*/
@Test
public void conjunctiveKeywordQueryTest() throws Exception {
@@ -251,7 +246,6 @@
/**
* Runs 50 random jaccard-based search queries with thresholds 1.0, 0.9,
* 0.8, 0.7, 0.6, 0.5. Tests the JaccardSearchModifier.
- *
*/
@Test
public void jaccardKeywordQueryTest() throws Exception {
diff --git a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
index 9999e95..53d2cab 100644
--- a/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
+++ b/hyracks-tests/hyracks-storage-am-invertedindex-test/src/test/java/edu/uci/ics/hyracks/storage/am/invertedindex/SearchTest.java
@@ -173,16 +173,12 @@
/**
* Runs a specified number of randomly picked strings from dataStrings as
* queries. We run each query, measure it's time, and print it's results.
- *
*/
private void runQueries(IInvertedIndexSearchModifier searchModifier, int numQueries) throws Exception {
-
rnd.setSeed(50);
-
InvertedIndexAccessor accessor = (InvertedIndexAccessor) invIndex.createAccessor(
NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
- InvertedIndexSearchPredicate searchPred = new InvertedIndexSearchPredicate(searchModifier);
-
+ InvertedIndexSearchPredicate searchPred = new InvertedIndexSearchPredicate(tokenizer, searchModifier);
for (int i = 0; i < numQueries; i++) {
int queryIndex = Math.abs(rnd.nextInt() % dataStrings.size());
@@ -236,7 +232,6 @@
/**
* Runs 5 random conjunctive search queries to test the
* ConjunctiveSearchModifier.
- *
*/
@Test
public void conjunctiveQueryTest() throws Exception {
@@ -247,7 +242,6 @@
/**
* Runs 5 random jaccard-based search queries with thresholds 0.9, 0.8, 0.7.
* Tests the JaccardSearchModifier.
- *
*/
@Test
public void jaccardQueryTest() throws Exception {
@@ -275,7 +269,6 @@
/**
* Runs 5 random edit-distance based search queries with thresholds 1, 2, 3.
* Tests the EditDistanceSearchModifier.
- *
*/
@Test
public void editDistanceQueryTest() throws Exception {
diff --git a/pom.xml b/pom.xml
index 98c3467..849d60c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1,3 +1,4 @@
+
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
<modelVersion>4.0.0</modelVersion>
<groupId>edu.uci.ics.hyracks</groupId>