Merge fullstack_hyracks_result_distribution again to fullstack_asterix_stabilization.
The tests pass this time, hopefully!
git-svn-id: https://hyracks.googlecode.com/svn/branches/fullstack_asterix_stabilization@3131 123451ca-8445-de46-9d55-352943316053
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
index 5234d2c..b8bdf3e 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
@@ -20,6 +20,7 @@
CLUSTER,
DATASOURCESCAN,
DISTINCT,
+ DISTRIBUTE_RESULT,
GROUP,
EMPTYTUPLESOURCE,
EXCHANGE,
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
index a969372..32cfb9a 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -7,6 +7,7 @@
BTREE_SEARCH,
STATS,
DATASOURCE_SCAN,
+ DISTRIBUTE_RESULT,
EMPTY_TUPLE_SOURCE,
EXTERNAL_GROUP_BY,
IN_MEMORY_HASH_JOIN,
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
index 899b633..82187e3 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
@@ -51,6 +51,10 @@
int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc)
throws AlgebricksException;
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink,
+ int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc, boolean ordered,
+ JobSpecification spec) throws AlgebricksException;
+
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getWriteResultRuntime(IDataSource<S> dataSource,
IOperatorSchema propagatedSchema, List<LogicalVariable> keys, LogicalVariable payLoadVar,
JobGenContext context, JobSpecification jobSpec) throws AlgebricksException;
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DistributeResultOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DistributeResultOperator.java
new file mode 100644
index 0000000..6ca6d87
--- /dev/null
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/DistributeResultOperator.java
@@ -0,0 +1,93 @@
+/*
+ * 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.algebricks.core.algebra.operators.logical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+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.metadata.IDataSink;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
+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;
+
+public class DistributeResultOperator extends AbstractLogicalOperator {
+ private List<Mutable<ILogicalExpression>> expressions;
+ private IDataSink dataSink;
+
+ public DistributeResultOperator(List<Mutable<ILogicalExpression>> expressions, IDataSink dataSink) {
+ this.expressions = expressions;
+ this.dataSink = dataSink;
+ }
+
+ public List<Mutable<ILogicalExpression>> getExpressions() {
+ return expressions;
+ }
+
+ public IDataSink getDataSink() {
+ return dataSink;
+ }
+
+ @Override
+ public LogicalOperatorTag getOperatorTag() {
+ return LogicalOperatorTag.DISTRIBUTE_RESULT;
+ }
+
+ @Override
+ public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+ return visitor.visitDistributeResultOperator(this, arg);
+ }
+
+ @Override
+ public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+ boolean modif = false;
+ for (int i = 0; i < expressions.size(); i++) {
+ boolean b = visitor.transform(expressions.get(i));
+ if (b) {
+ modif = true;
+ }
+ }
+ return modif;
+ }
+
+ @Override
+ public VariablePropagationPolicy getVariablePropagationPolicy() {
+ return VariablePropagationPolicy.ALL;
+ }
+
+ @Override
+ public boolean isMap() {
+ return false; // actually depends on the physical op.
+ }
+
+ @Override
+ public void recomputeSchema() {
+ schema = new ArrayList<LogicalVariable>();
+ schema.addAll(inputs.get(0).getValue().getSchema());
+ }
+
+ @Override
+ public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+ return createPropagatingAllInputsTypeEnvironment(ctx);
+ }
+
+}
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
index 0539cbe..1b4be1e 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
@@ -47,8 +47,10 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -64,7 +66,6 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -476,6 +477,14 @@
}
@Override
+ public Void visitDistributeResultOperator(DistributeResultOperator op, IOptimizationContext ctx)
+ throws AlgebricksException {
+ // propagateFDsAndEquivClasses(op, ctx);
+ setEmptyFDsEqClasses(op, ctx);
+ return null;
+ }
+
+ @Override
public Void visitWriteResultOperator(WriteResultOperator op, IOptimizationContext ctx) throws AlgebricksException {
// propagateFDsAndEquivClasses(op, ctx);
setEmptyFDsEqClasses(op, ctx);
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index b97597d..ac6d887 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -38,6 +38,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
@@ -425,6 +426,17 @@
}
@Override
+ public Boolean visitDistributeResultOperator(DistributeResultOperator op, ILogicalOperator arg)
+ throws AlgebricksException {
+ AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+ if (aop.getOperatorTag() != LogicalOperatorTag.DISTRIBUTE_RESULT)
+ return Boolean.FALSE;
+ DistributeResultOperator writeOpArg = (DistributeResultOperator) copyAndSubstituteVar(op, arg);
+ boolean isomorphic = VariableUtilities.varListEqualUnordered(op.getSchema(), writeOpArg.getSchema());
+ return isomorphic;
+ }
+
+ @Override
public Boolean visitWriteResultOperator(WriteResultOperator op, ILogicalOperator arg) throws AlgebricksException {
AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
if (aop.getOperatorTag() != LogicalOperatorTag.WRITE_RESULT)
@@ -762,6 +774,14 @@
}
@Override
+ public ILogicalOperator visitDistributeResultOperator(DistributeResultOperator op, Void arg)
+ throws AlgebricksException {
+ ArrayList<Mutable<ILogicalExpression>> newExpressions = new ArrayList<Mutable<ILogicalExpression>>();
+ deepCopyExpressionRefs(newExpressions, op.getExpressions());
+ return new DistributeResultOperator(newExpressions, op.getDataSink());
+ }
+
+ @Override
public ILogicalOperator visitWriteResultOperator(WriteResultOperator op, Void arg) throws AlgebricksException {
ArrayList<Mutable<ILogicalExpression>> newKeyExpressions = new ArrayList<Mutable<ILogicalExpression>>();
deepCopyExpressionRefs(newKeyExpressions, op.getKeyExpressions());
@@ -784,8 +804,8 @@
deepCopyExpressionRefs(newPrimaryKeyExpressions, op.getPrimaryKeyExpressions());
List<Mutable<ILogicalExpression>> newSecondaryKeyExpressions = new ArrayList<Mutable<ILogicalExpression>>();
deepCopyExpressionRefs(newSecondaryKeyExpressions, op.getSecondaryKeyExpressions());
- Mutable<ILogicalExpression> newFilterExpression = new MutableObject<ILogicalExpression>(((AbstractLogicalExpression)op.getFilterExpression())
- .cloneExpression());
+ Mutable<ILogicalExpression> newFilterExpression = new MutableObject<ILogicalExpression>(
+ ((AbstractLogicalExpression) op.getFilterExpression()).cloneExpression());
return new IndexInsertDeleteOperator(op.getDataSourceIndex(), newPrimaryKeyExpressions,
newSecondaryKeyExpressions, newFilterExpression, op.getOperation());
}
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
index 562bb4c..b9544c7 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
@@ -37,8 +37,10 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -54,7 +56,6 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -230,6 +231,13 @@
}
@Override
+ public Void visitDistributeResultOperator(DistributeResultOperator op, ILogicalOperator arg)
+ throws AlgebricksException {
+ mapVariablesStandard(op, arg);
+ return null;
+ }
+
+ @Override
public Void visitWriteResultOperator(WriteResultOperator op, ILogicalOperator arg) throws AlgebricksException {
mapVariablesStandard(op, arg);
return null;
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
index 9b2f5a0..8f1d686 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
@@ -29,8 +29,10 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -46,7 +48,6 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -240,6 +241,12 @@
}
@Override
+ public Void visitDistributeResultOperator(DistributeResultOperator op, IOptimizationContext arg)
+ throws AlgebricksException {
+ return null;
+ }
+
+ @Override
public Void visitWriteResultOperator(WriteResultOperator op, IOptimizationContext arg) throws AlgebricksException {
// TODO Auto-generated method stub
return null;
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
index 78b6801..31adcba 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
@@ -32,8 +32,10 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -49,7 +51,6 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -223,6 +224,11 @@
}
@Override
+ public Void visitDistributeResultOperator(DistributeResultOperator op, Void arg) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
public Void visitWriteResultOperator(WriteResultOperator op, Void arg) throws AlgebricksException {
return null;
}
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
index a759e35..cd0cee3 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
@@ -31,8 +31,10 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -48,7 +50,6 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -238,6 +239,12 @@
}
@Override
+ public Void visitDistributeResultOperator(DistributeResultOperator op, Void arg) throws AlgebricksException {
+ standardLayout(op);
+ return null;
+ }
+
+ @Override
public Void visitWriteResultOperator(WriteResultOperator op, Void arg) throws AlgebricksException {
standardLayout(op);
return null;
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
index 11e56ca..69fb3f8 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -33,8 +33,10 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -51,7 +53,6 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -335,6 +336,16 @@
}
@Override
+ public Void visitDistributeResultOperator(DistributeResultOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+ throws AlgebricksException {
+ for (Mutable<ILogicalExpression> e : op.getExpressions()) {
+ e.getValue().substituteVar(pair.first, pair.second);
+ }
+ substVarTypes(op, pair);
+ return null;
+ }
+
+ @Override
public Void visitWriteResultOperator(WriteResultOperator op, Pair<LogicalVariable, LogicalVariable> pair)
throws AlgebricksException {
op.getPayloadExpression().getValue().substituteVar(pair.first, pair.second);
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index 0ea9367..5361a19 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -32,6 +32,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
@@ -304,6 +305,14 @@
}
@Override
+ public Void visitDistributeResultOperator(DistributeResultOperator op, Void arg) {
+ for (Mutable<ILogicalExpression> expr : op.getExpressions()) {
+ expr.getValue().getUsedVariables(usedVariables);
+ }
+ return null;
+ }
+
+ @Override
public Void visitWriteResultOperator(WriteResultOperator op, Void arg) {
op.getPayloadExpression().getValue().getUsedVariables(usedVariables);
for (Mutable<ILogicalExpression> e : op.getKeyExpressions()) {
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
new file mode 100644
index 0000000..302d4d2
--- /dev/null
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/operators/physical/DistributeResultPOperator.java
@@ -0,0 +1,110 @@
+/*
+ * 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.algebricks.core.algebra.operators.physical;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+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.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+
+public class DistributeResultPOperator extends AbstractPhysicalOperator {
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.DISTRIBUTE_RESULT;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
+ ILogicalOperator op2 = op.getInputs().get(0).getValue();
+ deliveredProperties = op2.getDeliveredPhysicalProperties().clone();
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+ IPhysicalPropertiesVector reqdByParent) {
+ DistributeResultOperator write = (DistributeResultOperator) op;
+ IDataSink sink = write.getDataSink();
+ IPartitioningProperty pp = sink.getPartitioningProperty();
+ StructuralPropertiesVector[] r = new StructuralPropertiesVector[] { new StructuralPropertiesVector(pp, null) };
+ return new PhysicalRequirements(r, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+ }
+
+ @SuppressWarnings({ "rawtypes", "unchecked" })
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ DistributeResultOperator resultOp = (DistributeResultOperator) op;
+ IMetadataProvider mp = context.getMetadataProvider();
+
+ JobSpecification spec = builder.getJobSpec();
+
+ int[] columns = new int[resultOp.getExpressions().size()];
+ int i = 0;
+ for (Mutable<ILogicalExpression> exprRef : resultOp.getExpressions()) {
+ ILogicalExpression expr = exprRef.getValue();
+ if (expr.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+ throw new NotImplementedException("Only writing variable expressions is supported.");
+ }
+ VariableReferenceExpression varRef = (VariableReferenceExpression) expr;
+ LogicalVariable v = varRef.getVariableReference();
+ columns[i++] = inputSchemas[0].findVariable(v);
+ }
+ RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(
+ context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0], context);
+
+ IPrinterFactory[] pf = JobGenHelper.mkPrinterFactories(inputSchemas[0], context.getTypeEnvironment(op),
+ context, columns);
+
+ Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = mp.getResultHandleRuntime(
+ resultOp.getDataSink(), columns, pf, inputDesc, false, spec);
+
+ builder.contributeHyracksOperator(resultOp, runtimeAndConstraints.first);
+ builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
+ ILogicalOperator src = resultOp.getInputs().get(0).getValue();
+ builder.contributeGraphEdge(src, 0, resultOp, 0);
+ }
+}
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index a94c78e..fc0c433 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -30,8 +30,10 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -48,7 +50,6 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -164,6 +165,13 @@
}
@Override
+ public String visitDistributeResultOperator(DistributeResultOperator op, Integer indent) {
+ StringBuilder buffer = new StringBuilder();
+ addIndent(buffer, indent).append("distribute result ").append(op.getExpressions());
+ return buffer.toString();
+ }
+
+ @Override
public String visitWriteResultOperator(WriteResultOperator op, Integer indent) {
StringBuilder buffer = new StringBuilder();
addIndent(buffer, indent).append("load ").append(op.getDataSource()).append(" from ")
diff --git a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
index 6b5949e..23dac2a 100644
--- a/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/edu/uci/ics/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
@@ -20,8 +20,10 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DieOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -37,7 +39,6 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ScriptOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -97,6 +98,8 @@
public R visitWriteOperator(WriteOperator op, T arg) throws AlgebricksException;
+ public R visitDistributeResultOperator(DistributeResultOperator op, T arg) throws AlgebricksException;
+
public R visitWriteResultOperator(WriteResultOperator op, T arg) throws AlgebricksException;
public R visitInsertDeleteOperator(InsertDeleteOperator op, T tag) throws AlgebricksException;
diff --git a/algebricks/algebricks-data/src/main/java/edu/uci/ics/hyracks/algebricks/data/IResultSerializerFactoryProvider.java b/algebricks/algebricks-data/src/main/java/edu/uci/ics/hyracks/algebricks/data/IResultSerializerFactoryProvider.java
new file mode 100644
index 0000000..5e62612
--- /dev/null
+++ b/algebricks/algebricks-data/src/main/java/edu/uci/ics/hyracks/algebricks/data/IResultSerializerFactoryProvider.java
@@ -0,0 +1,37 @@
+/*
+ * 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.algebricks.data;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializerFactory;
+
+public interface IResultSerializerFactoryProvider extends Serializable {
+ /**
+ * Returns a result serializer factory
+ *
+ * @param fields
+ * - A position of the fields in the order it should be written in the output.
+ * @param printerFactories
+ * - A printer factory array to print the tuple containing different fields.
+ * @param writerFactory
+ * - A writer factory to write the serialized data to the print stream.
+ * @param inputRecordDesc
+ * - The record descriptor describing the input frame to be serialized.
+ * @return A new instance of result serialized appender.
+ */
+ public IResultSerializerFactory getAqlResultSerializerFactoryProvider(int[] fields,
+ IPrinterFactory[] printerFactories, IAWriterFactory writerFactory);
+}
diff --git a/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java b/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
index d105759..2981157 100644
--- a/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
+++ b/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/compiler/PigletCompiler.java
@@ -127,9 +127,9 @@
}
});
builder.setTypeTraitProvider(new ITypeTraitProvider() {
- public ITypeTraits getTypeTrait(Object type) {
- return null;
- }
+ public ITypeTraits getTypeTrait(Object type) {
+ return null;
+ }
});
builder.setPrinterProvider(PigletPrinterFactoryProvider.INSTANCE);
builder.setExpressionRuntimeProvider(new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter(
diff --git a/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java b/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
index d678803..15b290e 100644
--- a/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
+++ b/algebricks/algebricks-examples/piglet-example/src/main/java/edu/uci/ics/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
@@ -145,6 +145,13 @@
}
@Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink,
+ int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc, boolean ordered,
+ JobSpecification spec) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getWriteResultRuntime(
IDataSource<String> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
LogicalVariable payLoadVar, JobGenContext context, JobSpecification jobSpec) throws AlgebricksException {
diff --git a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
index 38cf96e..60a4fbb 100644
--- a/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/edu/uci/ics/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
@@ -38,6 +38,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AggregatePOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AssignPOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.DataSourceScanPOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.DistributeResultPOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.EmptyTupleSourcePOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.ExternalGroupByPOperator;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.InMemoryStableSortPOperator;
@@ -245,6 +246,10 @@
op.setPhysicalOperator(new SinkWritePOperator());
break;
}
+ case DISTRIBUTE_RESULT: {
+ op.setPhysicalOperator(new DistributeResultPOperator());
+ break;
+ }
case WRITE_RESULT: {
WriteResultOperator opLoad = (WriteResultOperator) op;
LogicalVariable payload;
@@ -267,8 +272,8 @@
List<LogicalVariable> secondaryKeys = new ArrayList<LogicalVariable>();
getKeys(opInsDel.getPrimaryKeyExpressions(), primaryKeys);
getKeys(opInsDel.getSecondaryKeyExpressions(), secondaryKeys);
- op.setPhysicalOperator(new IndexInsertDeletePOperator(primaryKeys, secondaryKeys,
- opInsDel.getFilterExpression(), opInsDel.getDataSourceIndex()));
+ op.setPhysicalOperator(new IndexInsertDeletePOperator(primaryKeys, secondaryKeys, opInsDel
+ .getFilterExpression(), opInsDel.getDataSourceIndex()));
break;
}
case SINK: {
diff --git a/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java
new file mode 100644
index 0000000..4f28c81
--- /dev/null
+++ b/algebricks/algebricks-runtime/src/main/java/edu/uci/ics/hyracks/algebricks/runtime/serializer/ResultSerializerFactoryProvider.java
@@ -0,0 +1,77 @@
+/*
+ * 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.algebricks.runtime.serializer;
+
+import java.io.PrintStream;
+import java.nio.BufferOverflowException;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.data.IAWriter;
+import edu.uci.ics.hyracks.algebricks.data.IAWriterFactory;
+import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
+import edu.uci.ics.hyracks.algebricks.data.IResultSerializerFactoryProvider;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class ResultSerializerFactoryProvider implements IResultSerializerFactoryProvider {
+ private static final long serialVersionUID = 1L;
+
+ public static final ResultSerializerFactoryProvider INSTANCE = new ResultSerializerFactoryProvider();
+
+ private ResultSerializerFactoryProvider() {
+ }
+
+ @Override
+ public IResultSerializerFactory getAqlResultSerializerFactoryProvider(final int[] fields,
+ final IPrinterFactory[] printerFactories, final IAWriterFactory writerFactory) {
+ return new IResultSerializerFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IResultSerializer createResultSerializer(RecordDescriptor inputRecordDesc, PrintStream printStream) {
+ final IAWriter writer = writerFactory.createWriter(fields, printStream, printerFactories,
+ inputRecordDesc);
+
+ return new IResultSerializer() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void init() throws HyracksDataException {
+ try {
+ writer.init();
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public boolean appendTuple(IFrameTupleAccessor tAccess, int tIdx) throws HyracksDataException {
+ try {
+ writer.printTuple(tAccess, tIdx);
+ } catch (BufferOverflowException e) {
+ return false;
+ } catch (AlgebricksException e) {
+ throw new HyracksDataException(e);
+ }
+ return true;
+ }
+ };
+ }
+ };
+ }
+}
diff --git a/algebricks/algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java b/algebricks/algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java
index b5c4b47..dad7cd0 100644
--- a/algebricks/algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java
+++ b/algebricks/algebricks-tests/src/test/java/edu/uci/ics/hyracks/algebricks/tests/util/AlgebricksHyracksIntegrationUtil.java
@@ -3,9 +3,9 @@
* 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.
@@ -55,6 +55,7 @@
ncConfig1.ccPort = TEST_HYRACKS_CC_CLUSTER_NET_PORT;
ncConfig1.clusterNetIPAddress = "127.0.0.1";
ncConfig1.dataIPAddress = "127.0.0.1";
+ ncConfig1.datasetIPAddress = "127.0.0.1";
ncConfig1.nodeId = NC1_ID;
nc1 = new NodeControllerService(ncConfig1);
nc1.start();
@@ -64,6 +65,7 @@
ncConfig2.ccPort = TEST_HYRACKS_CC_CLUSTER_NET_PORT;
ncConfig2.clusterNetIPAddress = "127.0.0.1";
ncConfig2.dataIPAddress = "127.0.0.1";
+ ncConfig2.datasetIPAddress = "127.0.0.1";
ncConfig2.nodeId = NC2_ID;
nc2 = new NodeControllerService(ncConfig2);
nc2.start();
@@ -85,4 +87,4 @@
hcc.waitForCompletion(jobId);
}
-}
\ No newline at end of file
+}
diff --git a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/perf/base/AbstractPerfTestSuiteClass.java b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/perf/base/AbstractPerfTestSuiteClass.java
index d9492d9..c882742 100644
--- a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/perf/base/AbstractPerfTestSuiteClass.java
+++ b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/perf/base/AbstractPerfTestSuiteClass.java
@@ -113,6 +113,7 @@
ncConfig.clusterNetIPAddress = ipAddress;
ncConfig.ccPort = clientPort;
ncConfig.dataIPAddress = "127.0.0.1";
+ ncConfig.datasetIPAddress = "127.0.0.1";
ncConfig.nodeId = "nc" + i;
NodeControllerService nc = new NodeControllerService(ncConfig);
nc.start();
diff --git a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/base/AbstractTestSuiteClass.java b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/base/AbstractTestSuiteClass.java
index 802777c..b258221 100644
--- a/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/base/AbstractTestSuiteClass.java
+++ b/hivesterix/hivesterix-dist/src/test/java/edu/uci/ics/hivesterix/test/base/AbstractTestSuiteClass.java
@@ -143,6 +143,7 @@
ncConfig.clusterNetIPAddress = ipAddress;
ncConfig.ccPort = netPort;
ncConfig.dataIPAddress = "127.0.0.1";
+ ncConfig.datasetIPAddress = "127.0.0.1";
ncConfig.nodeId = "nc" + i;
NodeControllerService nc = new NodeControllerService(ncConfig);
nc.start();
diff --git a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java
index 13526d8..daf6a7f 100644
--- a/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java
+++ b/hivesterix/hivesterix-runtime/src/main/java/edu/uci/ics/hivesterix/runtime/jobgen/HiveMetaDataProvider.java
@@ -95,6 +95,13 @@
}
@Override
+ public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink,
+ int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc, boolean ordered,
+ JobSpecification spec) throws AlgebricksException {
+ return null;
+ }
+
+ @Override
public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getWriteResultRuntime(IDataSource<S> arg0,
IOperatorSchema arg1, List<LogicalVariable> arg2, LogicalVariable arg3, JobGenContext arg4,
JobSpecification arg5) throws AlgebricksException {
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java
index a8f2fda..a4f0b29 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/channels/IInputChannel.java
@@ -16,7 +16,7 @@
import java.nio.ByteBuffer;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
public interface IInputChannel {
@@ -30,7 +30,7 @@
public void recycleBuffer(ByteBuffer buffer);
- public void open(IHyracksTaskContext ctx) throws HyracksDataException;
+ public void open(IHyracksCommonContext ctx) throws HyracksDataException;
public void close() throws HyracksDataException;
}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
index e34e60d..cd2b698 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceFunctions.java
@@ -17,6 +17,8 @@
import java.io.Serializable;
import java.util.EnumSet;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
@@ -30,6 +32,10 @@
CREATE_JOB,
GET_JOB_STATUS,
START_JOB,
+ GET_DATASET_DIRECTORY_SERIVICE_INFO,
+ GET_DATASET_RESULT_STATUS,
+ GET_DATASET_RECORD_DESCRIPTOR,
+ GET_DATASET_RESULT_LOCATIONS,
WAIT_FOR_COMPLETION,
GET_NODE_CONTROLLERS_INFO
}
@@ -156,6 +162,74 @@
}
}
+ public static class GetDatasetDirectoryServiceInfoFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.GET_DATASET_DIRECTORY_SERIVICE_INFO;
+ }
+ }
+
+ public static class GetDatasetResultStatusFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ public GetDatasetResultStatusFunction(JobId jobId, ResultSetId rsId) {
+ this.jobId = jobId;
+ this.rsId = rsId;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.GET_DATASET_RESULT_STATUS;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public ResultSetId getResultSetId() {
+ return rsId;
+ }
+ }
+
+ public static class GetDatasetResultLocationsFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final DatasetDirectoryRecord[] knownRecords;
+
+ public GetDatasetResultLocationsFunction(JobId jobId, ResultSetId rsId, DatasetDirectoryRecord[] knownRecords) {
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.knownRecords = knownRecords;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.GET_DATASET_RESULT_LOCATIONS;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public ResultSetId getResultSetId() {
+ return rsId;
+ }
+
+ public DatasetDirectoryRecord[] getKnownRecords() {
+ return knownRecords;
+ }
+ }
+
public static class WaitForCompletionFunction extends Function {
private static final long serialVersionUID = 1L;
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
index 4c06d42..2ab42c0 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksClientInterfaceRemoteProxy.java
@@ -17,6 +17,7 @@
import java.util.EnumSet;
import java.util.Map;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
@@ -76,6 +77,12 @@
}
@Override
+ public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception {
+ HyracksClientInterfaceFunctions.GetDatasetDirectoryServiceInfoFunction gddsf = new HyracksClientInterfaceFunctions.GetDatasetDirectoryServiceInfoFunction();
+ return (NetworkAddress) rpci.call(ipcHandle, gddsf);
+ }
+
+ @Override
public void waitForCompletion(JobId jobId) throws Exception {
HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf = new HyracksClientInterfaceFunctions.WaitForCompletionFunction(
jobId);
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
index 227524c..e0fafb0 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/HyracksConnection.java
@@ -26,6 +26,7 @@
import org.apache.http.impl.client.DefaultHttpClient;
import edu.uci.ics.hyracks.api.client.impl.JobSpecificationActivityClusterGraphGeneratorFactory;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
import edu.uci.ics.hyracks.api.job.JobFlag;
@@ -118,6 +119,10 @@
return hci.startJob(appName, JavaSerializationUtils.serialize(acggf), jobFlags);
}
+ public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception {
+ return hci.getDatasetDirectoryServiceInfo();
+ }
+
@Override
public void waitForCompletion(JobId jobId) throws Exception {
hci.waitForCompletion(jobId);
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
index bdbb544..6333c22 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientConnection.java
@@ -18,6 +18,7 @@
import java.util.EnumSet;
import java.util.Map;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
@@ -100,6 +101,14 @@
throws Exception;
/**
+ * Gets the IP Address and port for the DatasetDirectoryService wrapped in NetworkAddress
+ *
+ * @return {@link NetworkAddress}
+ * @throws Exception
+ */
+ public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception;
+
+ /**
* Waits until the specified job has completed, either successfully or has
* encountered a permanent failure.
*
@@ -123,4 +132,4 @@
* @throws Exception
*/
public ClusterTopology getClusterTopology() throws Exception;
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
index ef5906e..22b0a8f 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/IHyracksClientInterface.java
@@ -17,6 +17,7 @@
import java.util.EnumSet;
import java.util.Map;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
@@ -35,6 +36,8 @@
public JobId startJob(String appName, byte[] acggfBytes, EnumSet<JobFlag> jobFlags) throws Exception;
+ public NetworkAddress getDatasetDirectoryServiceInfo() throws Exception;
+
public void waitForCompletion(JobId jobId) throws Exception;
public Map<String, NodeControllerInfo> getNodeControllersInfo() throws Exception;
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
index fd9218a..73b5488 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/client/NodeControllerInfo.java
@@ -27,10 +27,14 @@
private final NetworkAddress netAddress;
- public NodeControllerInfo(String nodeId, NodeStatus status, NetworkAddress netAddress) {
+ private final NetworkAddress datasetNetworkAddress;
+
+ public NodeControllerInfo(String nodeId, NodeStatus status, NetworkAddress netAddress,
+ NetworkAddress datasetNetworkAddress) {
this.nodeId = nodeId;
this.status = status;
this.netAddress = netAddress;
+ this.datasetNetworkAddress = datasetNetworkAddress;
}
public String getNodeId() {
@@ -44,4 +48,8 @@
public NetworkAddress getNetworkAddress() {
return netAddress;
}
+
+ public NetworkAddress getDatasetNetworkAddress() {
+ return datasetNetworkAddress;
+ }
}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
index e964d66..a2ee977 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/context/IHyracksTaskContext.java
@@ -15,6 +15,7 @@
package edu.uci.ics.hyracks.api.context;
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
import edu.uci.ics.hyracks.api.job.IOperatorEnvironment;
import edu.uci.ics.hyracks.api.job.profiling.counters.ICounterContext;
@@ -28,5 +29,7 @@
public ICounterContext getCounterContext();
+ public IDatasetPartitionManager getDatasetPartitionManager();
+
public void sendApplicationMessageToCC(byte[] message, String nodeId) throws Exception;
}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializer.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializer.java
new file mode 100644
index 0000000..ba2ff9a
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializer.java
@@ -0,0 +1,38 @@
+/*
+ * 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.dataflow.value;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IResultSerializer extends Serializable {
+ /**
+ * Initializes the serializer.
+ */
+ public void init() throws HyracksDataException;
+
+ /**
+ * Method to serialize the result and append it to the provided output stream
+ *
+ * @param tAccess
+ * - A frame tuple accessor object that contains the original data to be serialized
+ * @param tIdx
+ * - Index of the tuple that should be serialized.
+ * @return true if the tuple was appended successfully, else false.
+ */
+ public boolean appendTuple(IFrameTupleAccessor tAccess, int tIdx) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializerFactory.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializerFactory.java
new file mode 100644
index 0000000..1fbf00f
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/IResultSerializerFactory.java
@@ -0,0 +1,29 @@
+/*
+ * 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.dataflow.value;
+
+import java.io.PrintStream;
+import java.io.Serializable;
+
+public interface IResultSerializerFactory extends Serializable {
+ /**
+ * Creates a result serialized appender
+ *
+ * @param printStream
+ * - A print stream object to which the serialized results will be written.
+ * @return A new instance of result serialized appender.
+ */
+ public IResultSerializer createResultSerializer(RecordDescriptor recordDesc, PrintStream printStream);
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/JSONSerializable.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/JSONSerializable.java
new file mode 100644
index 0000000..1eca502
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/dataflow/value/JSONSerializable.java
@@ -0,0 +1,27 @@
+/*
+ * 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.dataflow.value;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+public interface JSONSerializable {
+ /**
+ * Returns the JSON representation of the object.
+ *
+ * @return A new JSONObject instance representing this Java object.
+ */
+ public JSONObject toJSON() throws JSONException;
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
index 7c523f1..1fdff0f 100644
--- a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/job/JobSpecification.java
@@ -34,12 +34,15 @@
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;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
public class JobSpecification implements Serializable, IOperatorDescriptorRegistry, IConnectorDescriptorRegistry {
private static final long serialVersionUID = 1L;
private final List<OperatorDescriptorId> roots;
+ private final List<ResultSetId> resultSetIds;
+
private final Map<OperatorDescriptorId, IOperatorDescriptor> opMap;
private final Map<ConnectorDescriptorId, IConnectorDescriptor> connMap;
@@ -72,6 +75,7 @@
public JobSpecification() {
roots = new ArrayList<OperatorDescriptorId>();
+ resultSetIds = new ArrayList<ResultSetId>();
opMap = new HashMap<OperatorDescriptorId, IOperatorDescriptor>();
connMap = new HashMap<ConnectorDescriptorId, IConnectorDescriptor>();
opInputMap = new HashMap<OperatorDescriptorId, List<IConnectorDescriptor>>();
@@ -104,6 +108,10 @@
roots.add(op.getOperatorId());
}
+ public void addResultSetId(ResultSetId rsId) {
+ resultSetIds.add(rsId);
+ }
+
public void connect(IConnectorDescriptor conn, IOperatorDescriptor producerOp, int producerPort,
IOperatorDescriptor consumerOp, int consumerPort) {
insertIntoIndexedMap(opInputMap, consumerOp.getOperatorId(), consumerPort, conn);
@@ -208,6 +216,10 @@
return roots;
}
+ public List<ResultSetId> getResultSetIds() {
+ return resultSetIds;
+ }
+
public IConnectorPolicyAssignmentPolicy getConnectorPolicyAssignmentPolicy() {
return connectorPolicyAssignmentPolicy;
}
diff --git a/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/ResultSetPartitionId.java b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/ResultSetPartitionId.java
new file mode 100644
index 0000000..148a8a2
--- /dev/null
+++ b/hyracks/hyracks-api/src/main/java/edu/uci/ics/hyracks/api/partitions/ResultSetPartitionId.java
@@ -0,0 +1,87 @@
+/*
+ * 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.partitions;
+
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+public final class ResultSetPartitionId implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+
+ private final ResultSetId resultSetId;
+
+ private final int partition;
+
+ public ResultSetPartitionId(JobId jobId, ResultSetId resultSetId, int partition) {
+ this.jobId = jobId;
+ this.resultSetId = resultSetId;
+ this.partition = partition;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public ResultSetId getResultSetId() {
+ return resultSetId;
+ }
+
+ public int getPartition() {
+ return partition;
+ }
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + ((resultSetId == null) ? 0 : resultSetId.hashCode());
+ result = prime * result + ((jobId == null) ? 0 : jobId.hashCode());
+ result = prime * result + partition;
+ return result;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj)
+ return true;
+ if (obj == null)
+ return false;
+ if (getClass() != obj.getClass())
+ return false;
+ ResultSetPartitionId other = (ResultSetPartitionId) obj;
+ if (resultSetId == null) {
+ if (other.resultSetId != null)
+ return false;
+ } else if (!resultSetId.equals(other.resultSetId))
+ return false;
+ if (jobId == null) {
+ if (other.jobId != null)
+ return false;
+ } else if (!jobId.equals(other.jobId))
+ return false;
+ if (partition != other.partition)
+ return false;
+ return true;
+ }
+
+ @Override
+ public String toString() {
+ return jobId.toString() + ":" + resultSetId + ":" + partition;
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-client/pom.xml b/hyracks/hyracks-client/pom.xml
new file mode 100644
index 0000000..854a009
--- /dev/null
+++ b/hyracks/hyracks-client/pom.xml
@@ -0,0 +1,46 @@
+<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>
+ <artifactId>hyracks-client</artifactId>
+ <name>hyracks-client</name>
+ <parent>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </parent>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.7</source>
+ <target>1.7</target>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-api</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-net</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-comm</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-dataflow-common</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java
new file mode 100644
index 0000000..8be4a8c
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/DatasetClientContext.java
@@ -0,0 +1,44 @@
+/*
+ * 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.client.dataset;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+
+public class DatasetClientContext implements IHyracksCommonContext {
+ private final int frameSize;
+
+ public DatasetClientContext(int frameSize) {
+ this.frameSize = frameSize;
+ }
+
+ @Override
+ public int getFrameSize() {
+ return frameSize;
+ }
+
+ @Override
+ public IIOManager getIOManager() {
+ return null;
+ }
+
+ @Override
+ public ByteBuffer allocateFrame() {
+ return ByteBuffer.allocate(frameSize);
+ }
+
+}
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDataset.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDataset.java
new file mode 100644
index 0000000..6866e46
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDataset.java
@@ -0,0 +1,56 @@
+/*
+ * 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.client.dataset;
+
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceConnection;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.client.net.ClientNetworkManager;
+
+public class HyracksDataset implements IHyracksDataset {
+ private final IHyracksDatasetDirectoryServiceConnection datasetDirectoryServiceConnection;
+
+ private final ClientNetworkManager netManager;
+
+ private final DatasetClientContext datasetClientCtx;
+
+ public HyracksDataset(IHyracksClientConnection hcc, int frameSize, int nReaders) throws Exception {
+ NetworkAddress ddsAddress = hcc.getDatasetDirectoryServiceInfo();
+ datasetDirectoryServiceConnection = new HyracksDatasetDirectoryServiceConnection(new String(
+ ddsAddress.getIpAddress()), ddsAddress.getPort());
+
+ netManager = new ClientNetworkManager(nReaders);
+ netManager.start();
+
+ datasetClientCtx = new DatasetClientContext(frameSize);
+ }
+
+ @Override
+ public IHyracksDatasetReader createReader(JobId jobId, ResultSetId resultSetId) throws HyracksDataException {
+ IHyracksDatasetReader reader = null;
+ try {
+ reader = new HyracksDatasetReader(datasetDirectoryServiceConnection, netManager, datasetClientCtx, jobId,
+ resultSetId);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ }
+ return reader;
+ }
+}
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java
new file mode 100644
index 0000000..095fd7d
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceConnection.java
@@ -0,0 +1,53 @@
+/*
+ * 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.client.dataset;
+
+import java.net.InetSocketAddress;
+
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceConnection;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceInterface;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.api.RPCInterface;
+import edu.uci.ics.hyracks.ipc.impl.IPCSystem;
+import edu.uci.ics.hyracks.ipc.impl.JavaSerializationBasedPayloadSerializerDeserializer;
+
+//TODO(madhusudancs): Should this implementation be moved to edu.uci.ics.hyracks.client?
+public class HyracksDatasetDirectoryServiceConnection implements IHyracksDatasetDirectoryServiceConnection {
+ private final IPCSystem ipc;
+ private final IHyracksDatasetDirectoryServiceInterface ddsi;
+
+ public HyracksDatasetDirectoryServiceConnection(String ddsHost, int ddsPort) throws Exception {
+ RPCInterface rpci = new RPCInterface();
+ ipc = new IPCSystem(new InetSocketAddress(0), rpci, new JavaSerializationBasedPayloadSerializerDeserializer());
+ ipc.start();
+ IIPCHandle ddsIpchandle = ipc.getHandle(new InetSocketAddress(ddsHost, ddsPort));
+ this.ddsi = new HyracksDatasetDirectoryServiceInterfaceRemoteProxy(ddsIpchandle, rpci);
+ }
+
+ @Override
+ public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception {
+ return ddsi.getDatasetResultStatus(jobId, rsId);
+ }
+
+ @Override
+ public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
+ DatasetDirectoryRecord[] knownRecords) throws Exception {
+ return ddsi.getDatasetResultLocations(jobId, rsId, knownRecords);
+ }
+}
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java
new file mode 100644
index 0000000..47cdf97
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetDirectoryServiceInterfaceRemoteProxy.java
@@ -0,0 +1,51 @@
+/*
+ * 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.client.dataset;
+
+import edu.uci.ics.hyracks.api.client.HyracksClientInterfaceFunctions;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceInterface;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.ipc.api.IIPCHandle;
+import edu.uci.ics.hyracks.ipc.api.RPCInterface;
+
+//TODO(madhusudancs): Should this implementation be moved to edu.uci.ics.hyracks.client?
+public class HyracksDatasetDirectoryServiceInterfaceRemoteProxy implements IHyracksDatasetDirectoryServiceInterface {
+ private final IIPCHandle ipcHandle;
+
+ private final RPCInterface rpci;
+
+ public HyracksDatasetDirectoryServiceInterfaceRemoteProxy(IIPCHandle ipcHandle, RPCInterface rpci) {
+ this.ipcHandle = ipcHandle;
+ this.rpci = rpci;
+ }
+
+ @Override
+ public Status getDatasetResultStatus(JobId jobId, ResultSetId rsId) throws Exception {
+ HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction gdrlf = new HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction(
+ jobId, rsId);
+ return (Status) rpci.call(ipcHandle, gdrlf);
+ }
+
+ @Override
+ public DatasetDirectoryRecord[] getDatasetResultLocations(JobId jobId, ResultSetId rsId,
+ DatasetDirectoryRecord[] knownRecords) throws Exception {
+ HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction gdrlf = new HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction(
+ jobId, rsId, knownRecords);
+ return (DatasetDirectoryRecord[]) rpci.call(ipcHandle, gdrlf);
+ }
+}
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
new file mode 100644
index 0000000..78bcf20
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/dataset/HyracksDatasetReader.java
@@ -0,0 +1,255 @@
+/*
+ * 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.client.dataset;
+
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.net.UnknownHostException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.channels.IInputChannel;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IDatasetInputChannelMonitor;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetDirectoryServiceConnection;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.client.net.ClientNetworkManager;
+import edu.uci.ics.hyracks.comm.channels.DatasetNetworkInputChannel;
+
+// TODO(madhusudancs): Should this implementation be moved to edu.uci.ics.hyracks.client?
+public class HyracksDatasetReader implements IHyracksDatasetReader {
+ private static final Logger LOGGER = Logger.getLogger(HyracksDatasetReader.class.getName());
+
+ private final IHyracksDatasetDirectoryServiceConnection datasetDirectoryServiceConnection;
+
+ private final ClientNetworkManager netManager;
+
+ private final DatasetClientContext datasetClientCtx;
+
+ private JobId jobId;
+
+ private ResultSetId resultSetId;
+
+ private DatasetDirectoryRecord[] knownRecords;
+
+ private IDatasetInputChannelMonitor[] monitors;
+
+ private int lastReadPartition;
+
+ private IDatasetInputChannelMonitor lastMonitor;
+
+ private DatasetNetworkInputChannel resultChannel;
+
+ private static int NUM_READ_BUFFERS = 1;
+
+ public HyracksDatasetReader(IHyracksDatasetDirectoryServiceConnection datasetDirectoryServiceConnection,
+ ClientNetworkManager netManager, DatasetClientContext datasetClientCtx, JobId jobId, ResultSetId resultSetId)
+ throws Exception {
+ this.datasetDirectoryServiceConnection = datasetDirectoryServiceConnection;
+ this.netManager = netManager;
+ this.datasetClientCtx = datasetClientCtx;
+ this.jobId = jobId;
+ this.resultSetId = resultSetId;
+ knownRecords = null;
+ monitors = null;
+ lastReadPartition = -1;
+ lastMonitor = null;
+ resultChannel = null;
+ }
+
+ @Override
+ public Status getResultStatus() {
+ Status status = null;
+ try {
+ status = datasetDirectoryServiceConnection.getDatasetResultStatus(jobId, resultSetId);
+ } catch (Exception e) {
+ // TODO(madhusudancs): Decide what to do in case of error
+ }
+ return status;
+ }
+
+ @Override
+ public int read(ByteBuffer buffer) throws HyracksDataException {
+ ByteBuffer readBuffer;
+ int readSize = 0;
+
+ if (lastReadPartition == -1) {
+ while (knownRecords == null || knownRecords[0] == null) {
+ try {
+ knownRecords = datasetDirectoryServiceConnection.getDatasetResultLocations(jobId, resultSetId,
+ knownRecords);
+ lastReadPartition = 0;
+ resultChannel = new DatasetNetworkInputChannel(netManager,
+ getSocketAddress(knownRecords[lastReadPartition]), jobId, lastReadPartition,
+ NUM_READ_BUFFERS);
+ lastMonitor = getMonitor(lastReadPartition);
+ resultChannel.open(datasetClientCtx);
+ resultChannel.registerMonitor(lastMonitor);
+ } catch (HyracksException e) {
+ throw new HyracksDataException(e);
+ } catch (UnknownHostException e) {
+ throw new HyracksDataException(e);
+ } catch (Exception e) {
+ // Do nothing here.
+ }
+ }
+ }
+
+ while (readSize <= 0 && !((lastReadPartition == knownRecords.length - 1) && (lastMonitor.eosReached()))) {
+ synchronized (lastMonitor) {
+ while (lastMonitor.getNFramesAvailable() <= 0 && !lastMonitor.eosReached()) {
+ try {
+ lastMonitor.wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+
+ if (lastMonitor.getNFramesAvailable() <= 0 && lastMonitor.eosReached()) {
+ knownRecords[lastReadPartition].readEOS();
+ if ((lastReadPartition == knownRecords.length - 1)) {
+ break;
+ } else {
+ try {
+ lastReadPartition++;
+ while (knownRecords[lastReadPartition] == null) {
+ try {
+ knownRecords = datasetDirectoryServiceConnection.getDatasetResultLocations(jobId,
+ resultSetId, knownRecords);
+ } catch (Exception e) {
+ // Do nothing here.
+ }
+ }
+
+ resultChannel = new DatasetNetworkInputChannel(netManager,
+ getSocketAddress(knownRecords[lastReadPartition]), jobId, lastReadPartition,
+ NUM_READ_BUFFERS);
+ lastMonitor = getMonitor(lastReadPartition);
+ resultChannel.open(datasetClientCtx);
+ resultChannel.registerMonitor(lastMonitor);
+ } catch (HyracksException e) {
+ throw new HyracksDataException(e);
+ } catch (UnknownHostException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ } else {
+ readBuffer = resultChannel.getNextBuffer();
+ lastMonitor.notifyFrameRead();
+ if (readBuffer != null) {
+ buffer.put(readBuffer);
+ buffer.flip();
+ readSize = buffer.limit();
+ resultChannel.recycleBuffer(readBuffer);
+ }
+ }
+ }
+
+ return readSize;
+ }
+
+ private boolean nullExists(DatasetDirectoryRecord[] locations) {
+ if (locations == null) {
+ return true;
+ }
+ for (int i = 0; i < locations.length; i++) {
+ if (locations[i] == null) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private SocketAddress getSocketAddress(DatasetDirectoryRecord addr) throws UnknownHostException {
+ NetworkAddress netAddr = addr.getNetworkAddress();
+ return new InetSocketAddress(InetAddress.getByAddress(netAddr.getIpAddress()), netAddr.getPort());
+ }
+
+ private IDatasetInputChannelMonitor getMonitor(int partition) throws HyracksException {
+ if (knownRecords == null || knownRecords[partition] == null) {
+ throw new HyracksException("Accessing monitors before the obtaining the corresponding addresses.");
+ }
+ if (monitors == null) {
+ monitors = new DatasetInputChannelMonitor[knownRecords.length];
+ }
+ if (monitors[partition] == null) {
+ monitors[partition] = new DatasetInputChannelMonitor();
+ }
+ return monitors[partition];
+ }
+
+ private class DatasetInputChannelMonitor implements IDatasetInputChannelMonitor {
+ private final AtomicInteger nAvailableFrames;
+
+ private final AtomicBoolean eos;
+
+ private final AtomicBoolean failed;
+
+ public DatasetInputChannelMonitor() {
+ nAvailableFrames = new AtomicInteger(0);
+ eos = new AtomicBoolean(false);
+ failed = new AtomicBoolean(false);
+ }
+
+ @Override
+ public synchronized void notifyFailure(IInputChannel channel) {
+ failed.set(true);
+ notifyAll();
+ }
+
+ @Override
+ public synchronized void notifyDataAvailability(IInputChannel channel, int nFrames) {
+ nAvailableFrames.addAndGet(nFrames);
+ notifyAll();
+ }
+
+ @Override
+ public synchronized void notifyEndOfStream(IInputChannel channel) {
+ eos.set(true);
+ notifyAll();
+ }
+
+ @Override
+ public synchronized boolean eosReached() {
+ return eos.get();
+ }
+
+ @Override
+ public synchronized boolean failed() {
+ return failed.get();
+ }
+
+ @Override
+ public synchronized int getNFramesAvailable() {
+ return nAvailableFrames.get();
+ }
+
+ @Override
+ public synchronized void notifyFrameRead() {
+ nAvailableFrames.decrementAndGet();
+ }
+
+ }
+}
diff --git a/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/net/ClientNetworkManager.java b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/net/ClientNetworkManager.java
new file mode 100644
index 0000000..7aef8b9
--- /dev/null
+++ b/hyracks/hyracks-client/src/main/java/edu/uci/ics/hyracks/client/net/ClientNetworkManager.java
@@ -0,0 +1,56 @@
+/*
+ * 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.client.net;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+
+import edu.uci.ics.hyracks.comm.channels.IChannelConnectionFactory;
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MultiplexedConnection;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemux;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemuxPerformanceCounters;
+
+public class ClientNetworkManager implements IChannelConnectionFactory {
+ private static final int MAX_CONNECTION_ATTEMPTS = 5;
+
+ private final MuxDemux md;
+
+ public ClientNetworkManager(int nThreads) throws IOException {
+ /* This is a connect only socket and does not listen to any incoming connections, so pass null to
+ * localAddress and listener.
+ */
+ md = new MuxDemux(null, null, nThreads, MAX_CONNECTION_ATTEMPTS);
+ }
+
+ public void start() throws IOException {
+ md.start();
+ }
+
+ public void stop() {
+
+ }
+
+ public ChannelControlBlock connect(SocketAddress remoteAddress) throws InterruptedException, NetException {
+ MultiplexedConnection mConn = md.connect((InetSocketAddress) remoteAddress);
+ return mConn.openChannel();
+ }
+
+ public MuxDemuxPerformanceCounters getPerformanceCounters() {
+ return md.getPerformanceCounters();
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-comm/pom.xml b/hyracks/hyracks-comm/pom.xml
new file mode 100644
index 0000000..c3583699
--- /dev/null
+++ b/hyracks/hyracks-comm/pom.xml
@@ -0,0 +1,36 @@
+<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>
+ <artifactId>hyracks-comm</artifactId>
+ <name>hyracks-comm</name>
+ <parent>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </parent>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>2.0.2</version>
+ <configuration>
+ <source>1.7</source>
+ <target>1.7</target>
+ </configuration>
+ </plugin>
+ </plugins>
+ </build>
+ <dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-api</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-net</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </dependency>
+ </dependencies>
+</project>
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/DatasetNetworkInputChannel.java
similarity index 72%
copy from hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
copy to hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/DatasetNetworkInputChannel.java
index 1d5af84..fac2949 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/DatasetNetworkInputChannel.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.nc.net;
+package edu.uci.ics.hyracks.comm.channels;
import java.net.SocketAddress;
import java.nio.ByteBuffer;
@@ -23,21 +23,25 @@
import edu.uci.ics.hyracks.api.channels.IInputChannel;
import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
-public class NetworkInputChannel implements IInputChannel {
- private static final Logger LOGGER = Logger.getLogger(NetworkInputChannel.class.getName());
+public class DatasetNetworkInputChannel implements IInputChannel {
+ private static final Logger LOGGER = Logger.getLogger(DatasetNetworkInputChannel.class.getName());
- private final NetworkManager netManager;
+ static final int INITIAL_MESSAGE_SIZE = 20;
+
+ private final IChannelConnectionFactory netManager;
private final SocketAddress remoteAddress;
- private final PartitionId partitionId;
+ private final JobId jobId;
+
+ private final int partition;
private final Queue<ByteBuffer> fullQueue;
@@ -49,11 +53,12 @@
private Object attachment;
- public NetworkInputChannel(NetworkManager netManager, SocketAddress remoteAddress, PartitionId partitionId,
- int nBuffers) {
+ public DatasetNetworkInputChannel(IChannelConnectionFactory netManager, SocketAddress remoteAddress, JobId jobId,
+ int partition, int nBuffers) {
this.netManager = netManager;
this.remoteAddress = remoteAddress;
- this.partitionId = partitionId;
+ this.jobId = jobId;
+ this.partition = partition;
fullQueue = new ArrayDeque<ByteBuffer>(nBuffers);
this.nBuffers = nBuffers;
}
@@ -85,7 +90,7 @@
}
@Override
- public void open(IHyracksTaskContext ctx) throws HyracksDataException {
+ public void open(IHyracksCommonContext ctx) throws HyracksDataException {
try {
ccb = netManager.connect(remoteAddress);
} catch (Exception e) {
@@ -96,14 +101,13 @@
for (int i = 0; i < nBuffers; ++i) {
ccb.getReadInterface().getEmptyBufferAcceptor().accept(ctx.allocateFrame());
}
- ByteBuffer writeBuffer = ByteBuffer.allocate(NetworkManager.INITIAL_MESSAGE_SIZE);
- writeBuffer.putLong(partitionId.getJobId().getId());
- writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());
- writeBuffer.putInt(partitionId.getSenderIndex());
- writeBuffer.putInt(partitionId.getReceiverIndex());
+ ByteBuffer writeBuffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
+ writeBuffer.putLong(jobId.getId());
+ writeBuffer.putInt(partition);
writeBuffer.flip();
if (LOGGER.isLoggable(Level.FINE)) {
- LOGGER.fine("Sending partition request: " + partitionId + " on channel: " + ccb);
+ LOGGER.fine("Sending partition request for JobId: " + jobId + " partition: " + partition + " on channel: "
+ + ccb);
}
ccb.getWriteInterface().getFullBufferAcceptor().accept(writeBuffer);
ccb.getWriteInterface().getFullBufferAcceptor().close();
@@ -118,17 +122,17 @@
@Override
public void accept(ByteBuffer buffer) {
fullQueue.add(buffer);
- monitor.notifyDataAvailability(NetworkInputChannel.this, 1);
+ monitor.notifyDataAvailability(DatasetNetworkInputChannel.this, 1);
}
@Override
public void close() {
- monitor.notifyEndOfStream(NetworkInputChannel.this);
+ monitor.notifyEndOfStream(DatasetNetworkInputChannel.this);
}
@Override
public void error(int ecode) {
- monitor.notifyFailure(NetworkInputChannel.this);
+ monitor.notifyFailure(DatasetNetworkInputChannel.this);
}
}
diff --git a/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/IChannelConnectionFactory.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/IChannelConnectionFactory.java
new file mode 100644
index 0000000..33179ba
--- /dev/null
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/IChannelConnectionFactory.java
@@ -0,0 +1,24 @@
+/*
+ * 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.comm.channels;
+
+import java.net.SocketAddress;
+
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+
+public interface IChannelConnectionFactory {
+ public ChannelControlBlock connect(SocketAddress remoteAddress) throws InterruptedException, NetException;
+}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
similarity index 89%
rename from hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
rename to hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
index 1d5af84..aa37b16 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkInputChannel.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkInputChannel.java
@@ -12,7 +12,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.nc.net;
+package edu.uci.ics.hyracks.comm.channels;
import java.net.SocketAddress;
import java.nio.ByteBuffer;
@@ -23,7 +23,7 @@
import edu.uci.ics.hyracks.api.channels.IInputChannel;
import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
@@ -33,7 +33,9 @@
public class NetworkInputChannel implements IInputChannel {
private static final Logger LOGGER = Logger.getLogger(NetworkInputChannel.class.getName());
- private final NetworkManager netManager;
+ static final int INITIAL_MESSAGE_SIZE = 20;
+
+ private final IChannelConnectionFactory netManager;
private final SocketAddress remoteAddress;
@@ -49,8 +51,8 @@
private Object attachment;
- public NetworkInputChannel(NetworkManager netManager, SocketAddress remoteAddress, PartitionId partitionId,
- int nBuffers) {
+ public NetworkInputChannel(IChannelConnectionFactory netManager, SocketAddress remoteAddress,
+ PartitionId partitionId, int nBuffers) {
this.netManager = netManager;
this.remoteAddress = remoteAddress;
this.partitionId = partitionId;
@@ -85,7 +87,7 @@
}
@Override
- public void open(IHyracksTaskContext ctx) throws HyracksDataException {
+ public void open(IHyracksCommonContext ctx) throws HyracksDataException {
try {
ccb = netManager.connect(remoteAddress);
} catch (Exception e) {
@@ -96,7 +98,7 @@
for (int i = 0; i < nBuffers; ++i) {
ccb.getReadInterface().getEmptyBufferAcceptor().accept(ctx.allocateFrame());
}
- ByteBuffer writeBuffer = ByteBuffer.allocate(NetworkManager.INITIAL_MESSAGE_SIZE);
+ ByteBuffer writeBuffer = ByteBuffer.allocate(INITIAL_MESSAGE_SIZE);
writeBuffer.putLong(partitionId.getJobId().getId());
writeBuffer.putInt(partitionId.getConnectorDescriptorId().getId());
writeBuffer.putInt(partitionId.getSenderIndex());
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
similarity index 92%
rename from hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
rename to hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
index 9024e18..812a2de 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkOutputChannel.java
+++ b/hyracks/hyracks-comm/src/main/java/edu/uci/ics/hyracks/comm/channels/NetworkOutputChannel.java
@@ -12,14 +12,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package edu.uci.ics.hyracks.control.nc.net;
+package edu.uci.ics.hyracks.comm.channels;
import java.nio.ByteBuffer;
import java.util.ArrayDeque;
import java.util.Deque;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.net.buffers.IBufferAcceptor;
import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
@@ -40,9 +39,9 @@
ccb.getWriteInterface().setEmptyBufferAcceptor(new WriteEmptyBufferAcceptor());
}
- public void setTaskContext(IHyracksTaskContext ctx) {
+ public void setFrameSize(int frameSize) {
for (int i = 0; i < nBuffers; ++i) {
- emptyStack.push(ByteBuffer.allocateDirect(ctx.getFrameSize()));
+ emptyStack.push(ByteBuffer.allocateDirect(frameSize));
}
}
@@ -87,7 +86,7 @@
ccb.getWriteInterface().getFullBufferAcceptor().close();
}
- void abort() {
+ public void abort() {
ccb.getWriteInterface().getFullBufferAcceptor().error(1);
synchronized (NetworkOutputChannel.this) {
aborted = true;
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
index 5a33891..82457fe 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/ClusterControllerService.java
@@ -35,12 +35,17 @@
import edu.uci.ics.hyracks.api.client.ClusterControllerInfo;
import edu.uci.ics.hyracks.api.client.HyracksClientInterfaceFunctions;
import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.context.ICCContext;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
import edu.uci.ics.hyracks.api.topology.ClusterTopology;
import edu.uci.ics.hyracks.api.topology.TopologyDefinitionParser;
import edu.uci.ics.hyracks.control.cc.application.CCApplicationContext;
+import edu.uci.ics.hyracks.control.cc.dataset.DatasetDirectoryService;
import edu.uci.ics.hyracks.control.cc.job.JobRun;
import edu.uci.ics.hyracks.control.cc.web.WebServer;
import edu.uci.ics.hyracks.control.cc.work.ApplicationCreateWork;
@@ -48,17 +53,23 @@
import edu.uci.ics.hyracks.control.cc.work.ApplicationMessageWork;
import edu.uci.ics.hyracks.control.cc.work.ApplicationStartWork;
import edu.uci.ics.hyracks.control.cc.work.ApplicationStateChangeWork;
+import edu.uci.ics.hyracks.control.cc.work.GetDatasetDirectoryServiceInfoWork;
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.GetResultPartitionLocationsWork;
+import edu.uci.ics.hyracks.control.cc.work.GetResultStatusWork;
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;
import edu.uci.ics.hyracks.control.cc.work.RegisterNodeWork;
import edu.uci.ics.hyracks.control.cc.work.RegisterPartitionAvailibilityWork;
import edu.uci.ics.hyracks.control.cc.work.RegisterPartitionRequestWork;
+import edu.uci.ics.hyracks.control.cc.work.RegisterResultPartitionLocationWork;
import edu.uci.ics.hyracks.control.cc.work.RemoveDeadNodesWork;
import edu.uci.ics.hyracks.control.cc.work.ReportProfilesWork;
+import edu.uci.ics.hyracks.control.cc.work.ReportResultPartitionFailureWork;
+import edu.uci.ics.hyracks.control.cc.work.ReportResultPartitionWriteCompletionWork;
import edu.uci.ics.hyracks.control.cc.work.TaskCompleteWork;
import edu.uci.ics.hyracks.control.cc.work.TaskFailureWork;
import edu.uci.ics.hyracks.control.cc.work.UnregisterNodeWork;
@@ -115,6 +126,8 @@
private final DeadNodeSweeper sweeper;
+ private final IDatasetDirectoryService datasetDirectoryService;
+
private long jobCounter;
public ClusterControllerService(final CCConfig ccConfig) throws Exception {
@@ -162,6 +175,7 @@
}
};
sweeper = new DeadNodeSweeper();
+ datasetDirectoryService = new DatasetDirectoryService();
jobCounter = 0;
}
@@ -264,6 +278,10 @@
return clusterIPC;
}
+ public NetworkAddress getDatasetDirectoryServiceInfo() {
+ return new NetworkAddress(ccConfig.clientNetIpAddress.getBytes(), ccConfig.clientNetPort);
+ }
+
private class DeadNodeSweeper extends TimerTask {
@Override
public void run() {
@@ -271,6 +289,10 @@
}
}
+ public IDatasetDirectoryService getDatasetDirectoryService() {
+ return datasetDirectoryService;
+ }
+
private class HyracksClientInterfaceIPCI implements IIPCI {
@Override
public void deliverIncomingMessage(IIPCHandle handle, long mid, long rmid, Object payload, Exception exception) {
@@ -321,6 +343,27 @@
return;
}
+ case GET_DATASET_DIRECTORY_SERIVICE_INFO: {
+ workQueue.schedule(new GetDatasetDirectoryServiceInfoWork(ClusterControllerService.this,
+ new IPCResponder<NetworkAddress>(handle, mid)));
+ return;
+ }
+
+ case GET_DATASET_RESULT_STATUS: {
+ HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction gdrlf = (HyracksClientInterfaceFunctions.GetDatasetResultStatusFunction) fn;
+ workQueue.schedule(new GetResultStatusWork(ClusterControllerService.this, gdrlf.getJobId(), gdrlf
+ .getResultSetId(), new IPCResponder<Status>(handle, mid)));
+ return;
+ }
+
+ case GET_DATASET_RESULT_LOCATIONS: {
+ HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction gdrlf = (HyracksClientInterfaceFunctions.GetDatasetResultLocationsFunction) fn;
+ workQueue.schedule(new GetResultPartitionLocationsWork(ClusterControllerService.this, gdrlf
+ .getJobId(), gdrlf.getResultSetId(), gdrlf.getKnownRecords(),
+ new IPCResponder<DatasetDirectoryRecord[]>(handle, mid)));
+ return;
+ }
+
case WAIT_FOR_COMPLETION: {
HyracksClientInterfaceFunctions.WaitForCompletionFunction wfcf = (HyracksClientInterfaceFunctions.WaitForCompletionFunction) fn;
workQueue.schedule(new WaitForJobCompletionWork(ClusterControllerService.this, wfcf.getJobId(),
@@ -416,6 +459,28 @@
return;
}
+ case REGISTER_RESULT_PARTITION_LOCATION: {
+ CCNCFunctions.RegisterResultPartitionLocationFunction rrplf = (CCNCFunctions.RegisterResultPartitionLocationFunction) fn;
+ workQueue.schedule(new RegisterResultPartitionLocationWork(ClusterControllerService.this, rrplf
+ .getJobId(), rrplf.getResultSetId(), rrplf.getOrderedResult(), rrplf.getPartition(), rrplf
+ .getNPartitions(), rrplf.getNetworkAddress()));
+ return;
+ }
+
+ case REPORT_RESULT_PARTITION_WRITE_COMPLETION: {
+ CCNCFunctions.ReportResultPartitionWriteCompletionFunction rrplf = (CCNCFunctions.ReportResultPartitionWriteCompletionFunction) fn;
+ workQueue.schedule(new ReportResultPartitionWriteCompletionWork(ClusterControllerService.this,
+ rrplf.getJobId(), rrplf.getResultSetId(), rrplf.getPartition()));
+ return;
+ }
+
+ case REPORT_RESULT_PARTITION_FAILURE: {
+ CCNCFunctions.ReportResultPartitionFailureFunction rrplf = (CCNCFunctions.ReportResultPartitionFailureFunction) fn;
+ workQueue.schedule(new ReportResultPartitionFailureWork(ClusterControllerService.this, rrplf
+ .getJobId(), rrplf.getResultSetId(), rrplf.getPartition()));
+ return;
+ }
+
case APPLICATION_STATE_CHANGE_RESPONSE: {
CCNCFunctions.ApplicationStateChangeResponseFunction astrf = (CCNCFunctions.ApplicationStateChangeResponseFunction) fn;
workQueue.schedule(new ApplicationStateChangeWork(ClusterControllerService.this, astrf));
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
index c17acd0..c96a319 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/NodeControllerState.java
@@ -41,6 +41,8 @@
private final NetworkAddress dataPort;
+ private final NetworkAddress datasetPort;
+
private final Set<JobId> activeJobIds;
private final String osName;
@@ -107,6 +109,14 @@
private final long[] netSignalingBytesWritten;
+ private final long[] datasetNetPayloadBytesRead;
+
+ private final long[] datasetNetPayloadBytesWritten;
+
+ private final long[] datasetNetSignalingBytesRead;
+
+ private final long[] datasetNetSignalingBytesWritten;
+
private final long[] ipcMessagesSent;
private final long[] ipcMessageBytesSent;
@@ -123,6 +133,7 @@
this.nodeController = nodeController;
ncConfig = reg.getNCConfig();
dataPort = reg.getDataPort();
+ datasetPort = reg.getDatasetPort();
activeJobIds = new HashSet<JobId>();
osName = reg.getOSName();
@@ -164,6 +175,10 @@
netPayloadBytesWritten = new long[RRD_SIZE];
netSignalingBytesRead = new long[RRD_SIZE];
netSignalingBytesWritten = new long[RRD_SIZE];
+ datasetNetPayloadBytesRead = new long[RRD_SIZE];
+ datasetNetPayloadBytesWritten = new long[RRD_SIZE];
+ datasetNetSignalingBytesRead = new long[RRD_SIZE];
+ datasetNetSignalingBytesWritten = new long[RRD_SIZE];
ipcMessagesSent = new long[RRD_SIZE];
ipcMessageBytesSent = new long[RRD_SIZE];
ipcMessagesReceived = new long[RRD_SIZE];
@@ -196,6 +211,10 @@
netPayloadBytesWritten[rrdPtr] = hbData.netPayloadBytesWritten;
netSignalingBytesRead[rrdPtr] = hbData.netSignalingBytesRead;
netSignalingBytesWritten[rrdPtr] = hbData.netSignalingBytesWritten;
+ datasetNetPayloadBytesRead[rrdPtr] = hbData.datasetNetPayloadBytesRead;
+ datasetNetPayloadBytesWritten[rrdPtr] = hbData.datasetNetPayloadBytesWritten;
+ datasetNetSignalingBytesRead[rrdPtr] = hbData.datasetNetSignalingBytesRead;
+ datasetNetSignalingBytesWritten[rrdPtr] = hbData.datasetNetSignalingBytesWritten;
ipcMessagesSent[rrdPtr] = hbData.ipcMessagesSent;
ipcMessageBytesSent[rrdPtr] = hbData.ipcMessageBytesSent;
ipcMessagesReceived[rrdPtr] = hbData.ipcMessagesReceived;
@@ -227,6 +246,10 @@
return dataPort;
}
+ public NetworkAddress getDatasetPort() {
+ return datasetPort;
+ }
+
public JSONObject toSummaryJSON() throws JSONException {
JSONObject o = new JSONObject();
o.put("node-id", ncConfig.nodeId);
@@ -271,6 +294,10 @@
o.put("net-payload-bytes-written", netPayloadBytesWritten);
o.put("net-signaling-bytes-read", netSignalingBytesRead);
o.put("net-signaling-bytes-written", netSignalingBytesWritten);
+ o.put("dataset-net-payload-bytes-read", datasetNetPayloadBytesRead);
+ o.put("dataset-net-payload-bytes-written", datasetNetPayloadBytesWritten);
+ o.put("dataset-net-signaling-bytes-read", datasetNetSignalingBytesRead);
+ o.put("dataset-net-signaling-bytes-written", datasetNetSignalingBytesWritten);
o.put("ipc-messages-sent", ipcMessagesSent);
o.put("ipc-message-bytes-sent", ipcMessageBytesSent);
o.put("ipc-messages-received", ipcMessagesReceived);
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/dataset/DatasetDirectoryService.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/dataset/DatasetDirectoryService.java
new file mode 100644
index 0000000..13d0c30
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/dataset/DatasetDirectoryService.java
@@ -0,0 +1,241 @@
+/*
+ * 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.dataset;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+
+/**
+ * TODO(madhusudancs): The potential perils of this global dataset directory service implementation is that, the jobs
+ * location information is never evicted from the memory and the memory usage grows as the number of jobs in the system
+ * grows. What we should possibly do is, add an API call for the client to say that it received everything it has to for
+ * the job (after it receives all the results) completely. Then we can just get rid of the location information for that
+ * job.
+ */
+public class DatasetDirectoryService implements IDatasetDirectoryService {
+ private final Map<JobId, Map<ResultSetId, ResultSetMetaData>> jobResultLocationsMap;
+
+ public DatasetDirectoryService() {
+ jobResultLocationsMap = new HashMap<JobId, Map<ResultSetId, ResultSetMetaData>>();
+ }
+
+ @Override
+ public synchronized void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult,
+ int partition, int nPartitions, NetworkAddress networkAddress) {
+ Map<ResultSetId, ResultSetMetaData> rsMap = jobResultLocationsMap.get(jobId);
+ if (rsMap == null) {
+ rsMap = new HashMap<ResultSetId, ResultSetMetaData>();
+ jobResultLocationsMap.put(jobId, rsMap);
+ }
+
+ ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+ if (resultSetMetaData == null) {
+ resultSetMetaData = new ResultSetMetaData(orderedResult, new DatasetDirectoryRecord[nPartitions]);
+ rsMap.put(rsId, resultSetMetaData);
+ }
+
+ DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
+ if (records[partition] == null) {
+ records[partition] = new DatasetDirectoryRecord();
+ }
+ records[partition].setNetworkAddress(networkAddress);
+ records[partition].start();
+ notifyAll();
+ }
+
+ @Override
+ public synchronized void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) {
+ DatasetDirectoryRecord ddr = getDatasetDirectoryRecord(jobId, rsId, partition);
+ ddr.writeEOS();
+ }
+
+ @Override
+ public synchronized void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition) {
+ DatasetDirectoryRecord ddr = getDatasetDirectoryRecord(jobId, rsId, partition);
+ ddr.fail();
+ }
+
+ @Override
+ public synchronized Status getResultStatus(JobId jobId, ResultSetId rsId) throws HyracksDataException {
+ Map<ResultSetId, ResultSetMetaData> rsMap;
+ while ((rsMap = jobResultLocationsMap.get(jobId)) == null) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+ if (resultSetMetaData == null || resultSetMetaData.getRecords() == null) {
+ throw new HyracksDataException("ResultSet locations uninitialized when it is expected to be initialized.");
+ }
+ DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
+
+ ArrayList<Status> statuses = new ArrayList<Status>(records.length);
+ for (int i = 0; i < records.length; i++) {
+ statuses.add(records[i].getStatus());
+ }
+
+ // Default status is idle
+ Status status = Status.IDLE;
+ if (statuses.contains(Status.FAILED)) {
+ // Even if there is at least one failed entry we should return failed status.
+ return Status.FAILED;
+ } else if (statuses.contains(Status.RUNNING)) {
+ // If there are not failed entry and if there is at least one running entry we should return running status.
+ return Status.RUNNING;
+ } else {
+ // If each and every partition has reported success do we report success as the status.
+ int successCount = 0;
+ for (int i = 0; i < statuses.size(); i++) {
+ if (statuses.get(i) == Status.SUCCESS) {
+ successCount++;
+ }
+ }
+ if (successCount == statuses.size()) {
+ return Status.SUCCESS;
+ }
+ }
+ return status;
+ }
+
+ @Override
+ public synchronized DatasetDirectoryRecord[] getResultPartitionLocations(JobId jobId, ResultSetId rsId,
+ DatasetDirectoryRecord[] knownRecords) throws HyracksDataException {
+ DatasetDirectoryRecord[] newRecords;
+ while ((newRecords = updatedRecords(jobId, rsId, knownRecords)) == null) {
+ try {
+ wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ return newRecords;
+ }
+
+ public DatasetDirectoryRecord getDatasetDirectoryRecord(JobId jobId, ResultSetId rsId, int partition) {
+ Map<ResultSetId, ResultSetMetaData> rsMap = jobResultLocationsMap.get(jobId);
+ ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+ DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
+ return records[partition];
+ }
+
+ /**
+ * Compares the records already known by the client for the given job's result set id with the records that the
+ * dataset directory service knows and if there are any newly discovered records returns a whole array with the
+ * new records filled in.
+ * This method has a very convoluted logic. Here is the explanation of how it works.
+ * If the ordering constraint has to be enforced, the method obtains the first null record in the known records in
+ * the order of the partitions. It always traverses the array in the first to last order!
+ * If known records array or the first element in that array is null in the but the record for that partition now
+ * known to the directory service, the method fills in that record in the array and returns the array back.
+ * However, if the first known null record is not a first element in the array, by induction, all the previous
+ * known records should be known already be known to client and none of the records for the partitions ahead is
+ * known by the client yet. So, we check if the client has reached the end of stream for the partition corresponding
+ * to the record before the first known null record, i.e. the last known non-null record. If not, we just return
+ * null because we cannot expose any new locations until the client reaches end of stream for the last known record.
+ * If the client has reached the end of stream record for the last known non-null record, we check if the next record
+ * is discovered by the dataset directory service and if so, we fill it in the records array and return it back or
+ * send null otherwise.
+ * If the ordering is not required, we are free to return any newly discovered records back, so we just check if
+ * arrays are equal and if they are not we send the entire new updated array.
+ *
+ * @param jobId
+ * - Id of the job for which the directory records should be retrieved.
+ * @param rsId
+ * - Id of the result set for which the directory records should be retrieved.
+ * @param knownRecords
+ * - An array of directory records that the client is already aware of.
+ * @return
+ * - Returns null if there aren't any newly discovered partitions enforcing the ordering constraint
+ * @throws HyracksDataException
+ * TODO(madhusudancs): Think about caching (and still be stateless) instead of this ugly O(n) iterations for
+ * every check. This already looks very expensive.
+ */
+ private DatasetDirectoryRecord[] updatedRecords(JobId jobId, ResultSetId rsId, DatasetDirectoryRecord[] knownRecords)
+ throws HyracksDataException {
+ Map<ResultSetId, ResultSetMetaData> rsMap = jobResultLocationsMap.get(jobId);
+ if (rsMap == null) {
+ return null;
+ }
+
+ ResultSetMetaData resultSetMetaData = rsMap.get(rsId);
+ if (resultSetMetaData == null || resultSetMetaData.getRecords() == null) {
+ throw new HyracksDataException("ResultSet locations uninitialized when it is expected to be initialized.");
+ }
+
+ boolean ordered = resultSetMetaData.getOrderedResult();
+ DatasetDirectoryRecord[] records = resultSetMetaData.getRecords();
+ /* If ordering is required, we should expose the dataset directory records only in the order, otherwise
+ * we can simply check if there are any newly discovered records and send the whole array back if there are.
+ */
+ if (ordered) {
+ // Iterate over the known records and find the last record which is not null.
+ int i = 0;
+ for (i = 0; i < records.length; i++) {
+ if (knownRecords == null) {
+ if (records[0] != null) {
+ knownRecords = new DatasetDirectoryRecord[records.length];
+ knownRecords[0] = records[0];
+ return knownRecords;
+ }
+ return null;
+ }
+ if (knownRecords[i] == null) {
+ if ((i == 0 || knownRecords[i - 1].hasReachedReadEOS()) && records[i] != null) {
+ knownRecords[i] = records[i];
+ return knownRecords;
+ }
+ return null;
+ }
+ }
+ } else {
+ if (!Arrays.equals(records, knownRecords)) {
+ return records;
+ }
+ }
+ return null;
+ }
+
+ private class ResultSetMetaData {
+ private final boolean ordered;
+
+ private final DatasetDirectoryRecord[] records;
+
+ public ResultSetMetaData(boolean ordered, DatasetDirectoryRecord[] records) {
+ this.ordered = ordered;
+ this.records = records;
+ }
+
+ public boolean getOrderedResult() {
+ return ordered;
+ }
+
+ public DatasetDirectoryRecord[] getRecords() {
+ return records;
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetDatasetDirectoryServiceInfoWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetDatasetDirectoryServiceInfoWork.java
new file mode 100644
index 0000000..3ac6acc
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetDatasetDirectoryServiceInfoWork.java
@@ -0,0 +1,41 @@
+/*
+ * 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 edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+
+public class GetDatasetDirectoryServiceInfoWork extends SynchronizableWork {
+ private final ClusterControllerService ccs;
+
+ private final IResultCallback<NetworkAddress> callback;
+
+ public GetDatasetDirectoryServiceInfoWork(ClusterControllerService ccs, IResultCallback<NetworkAddress> callback) {
+ this.ccs = ccs;
+ this.callback = callback;
+ }
+
+ @Override
+ public void doRun() {
+ try {
+ NetworkAddress addr = ccs.getDatasetDirectoryServiceInfo();
+ callback.setValue(addr);
+ } catch (Exception e) {
+ callback.setException(e);
+ }
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
index 2f23a2c..a787b9f 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetNodeControllersInfoWork.java
@@ -39,7 +39,8 @@
Map<String, NodeControllerInfo> result = new LinkedHashMap<String, NodeControllerInfo>();
Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
for (Map.Entry<String, NodeControllerState> e : nodeMap.entrySet()) {
- result.put(e.getKey(), new NodeControllerInfo(e.getKey(), NodeStatus.ALIVE, e.getValue().getDataPort()));
+ result.put(e.getKey(), new NodeControllerInfo(e.getKey(), NodeStatus.ALIVE, e.getValue().getDataPort(), e
+ .getValue().getDatasetPort()));
}
callback.setValue(result);
}
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultPartitionLocationsWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultPartitionLocationsWork.java
new file mode 100644
index 0000000..fd1d418
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultPartitionLocationsWork.java
@@ -0,0 +1,62 @@
+/*
+ * 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 edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord;
+import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+
+public class GetResultPartitionLocationsWork extends SynchronizableWork {
+ private final ClusterControllerService ccs;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final DatasetDirectoryRecord[] knownRecords;
+
+ private final IResultCallback<DatasetDirectoryRecord[]> callback;
+
+ public GetResultPartitionLocationsWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId,
+ DatasetDirectoryRecord[] knownRecords, IResultCallback<DatasetDirectoryRecord[]> callback) {
+ this.ccs = ccs;
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.knownRecords = knownRecords;
+ this.callback = callback;
+ }
+
+ @Override
+ public void doRun() {
+ final IDatasetDirectoryService dds = ccs.getDatasetDirectoryService();
+ ccs.getExecutor().execute(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ DatasetDirectoryRecord[] partitionLocations = dds.getResultPartitionLocations(jobId, rsId,
+ knownRecords);
+ callback.setValue(partitionLocations);
+ } catch (HyracksDataException e) {
+ callback.setException(e);
+ }
+ }
+ });
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultStatusWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultStatusWork.java
new file mode 100644
index 0000000..d2dadf5
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/GetResultStatusWork.java
@@ -0,0 +1,56 @@
+/*
+ * 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 edu.uci.ics.hyracks.api.dataset.DatasetDirectoryRecord.Status;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.IResultCallback;
+import edu.uci.ics.hyracks.control.common.work.SynchronizableWork;
+
+public class GetResultStatusWork extends SynchronizableWork {
+ private final ClusterControllerService ccs;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final IResultCallback<Status> callback;
+
+ public GetResultStatusWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId,
+ IResultCallback<Status> callback) {
+ this.ccs = ccs;
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.callback = callback;
+ }
+
+ @Override
+ public void doRun() {
+ try {
+ Status status = ccs.getDatasetDirectoryService().getResultStatus(jobId, rsId);
+ callback.setValue(status);
+ } catch (HyracksDataException e) {
+ callback.setException(e);
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "JobId@" + jobId + " ResultSetId@" + rsId;
+ }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
index b6a33cd..b062d33 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/JobStartWork.java
@@ -16,6 +16,7 @@
import java.util.EnumSet;
+import edu.uci.ics.hyracks.api.dataset.IDatasetDirectoryService;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGenerator;
import edu.uci.ics.hyracks.api.job.IActivityClusterGraphGeneratorFactory;
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java
new file mode 100644
index 0000000..f86e924
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/RegisterResultPartitionLocationWork.java
@@ -0,0 +1,60 @@
+/*
+ * 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 edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+
+public class RegisterResultPartitionLocationWork extends AbstractWork {
+ private final ClusterControllerService ccs;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final boolean orderedResult;
+
+ private final int partition;
+
+ private final int nPartitions;
+
+ private final NetworkAddress networkAddress;
+
+ public RegisterResultPartitionLocationWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId,
+ boolean orderedResult, int partition, int nPartitions, NetworkAddress networkAddress) {
+ this.ccs = ccs;
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.orderedResult = orderedResult;
+ this.partition = partition;
+ this.nPartitions = nPartitions;
+ this.networkAddress = networkAddress;
+ }
+
+ @Override
+ public void run() {
+ ccs.getDatasetDirectoryService().registerResultPartitionLocation(jobId, rsId, orderedResult, partition,
+ nPartitions, networkAddress);
+ }
+
+ @Override
+ public String toString() {
+ return "JobId@" + jobId + " ResultSetId@" + rsId + " Partition@" + partition + " NPartitions@" + nPartitions
+ + " ResultPartitionLocation@" + networkAddress + " OrderedResult@" + orderedResult;
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionFailureWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionFailureWork.java
new file mode 100644
index 0000000..4aea41e
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionFailureWork.java
@@ -0,0 +1,47 @@
+/*
+ * 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 edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+
+public class ReportResultPartitionFailureWork extends AbstractWork {
+ private final ClusterControllerService ccs;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final int partition;
+
+ public ReportResultPartitionFailureWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId, int partition) {
+ this.ccs = ccs;
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.partition = partition;
+ }
+
+ @Override
+ public void run() {
+ ccs.getDatasetDirectoryService().reportResultPartitionFailure(jobId, rsId, partition);
+ }
+
+ @Override
+ public String toString() {
+ return "JobId@" + jobId + " ResultSetId@" + rsId + " Partition@" + partition;
+ }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionWriteCompletionWork.java b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionWriteCompletionWork.java
new file mode 100644
index 0000000..313b730
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/java/edu/uci/ics/hyracks/control/cc/work/ReportResultPartitionWriteCompletionWork.java
@@ -0,0 +1,48 @@
+/*
+ * 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 edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
+import edu.uci.ics.hyracks.control.common.work.AbstractWork;
+
+public class ReportResultPartitionWriteCompletionWork extends AbstractWork {
+ private final ClusterControllerService ccs;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final int partition;
+
+ public ReportResultPartitionWriteCompletionWork(ClusterControllerService ccs, JobId jobId, ResultSetId rsId,
+ int partition) {
+ this.ccs = ccs;
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.partition = partition;
+ }
+
+ @Override
+ public void run() {
+ ccs.getDatasetDirectoryService().reportResultPartitionWriteCompletion(jobId, rsId, partition);
+ }
+
+ @Override
+ public String toString() {
+ return "JobId@" + jobId + " ResultSetId@" + rsId + " Partition@" + partition;
+ }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-cc/src/main/resources/static/javascript/adminconsole/NodeDetailsPage.js b/hyracks/hyracks-control/hyracks-control-cc/src/main/resources/static/javascript/adminconsole/NodeDetailsPage.js
index ff9d8a0..3fc46ff 100644
--- a/hyracks/hyracks-control/hyracks-control-cc/src/main/resources/static/javascript/adminconsole/NodeDetailsPage.js
+++ b/hyracks/hyracks-control/hyracks-control-cc/src/main/resources/static/javascript/adminconsole/NodeDetailsPage.js
@@ -58,6 +58,10 @@
var netPayloadBytesWritten = result['net-payload-bytes-written'];
var netSignalingBytesRead = result['net-signaling-bytes-read'];
var netSignalingBytesWritten = result['net-signaling-bytes-written'];
+ var datasetNetPayloadBytesRead = result['dataset-net-payload-bytes-read'];
+ var datasetNetPayloadBytesWritten = result['dataset-net-payload-bytes-written'];
+ var datasetNetSignalingBytesRead = result['dataset-net-signaling-bytes-read'];
+ var datasetNetSignalingBytesWritten = result['dataset-net-signaling-bytes-written'];
var ipcMessagesSent = result['ipc-messages-sent'];
var ipcMessageBytesSent = result['ipc-message-bytes-sent'];
var ipcMessagesReceived = result['ipc-messages-received'];
@@ -117,9 +121,13 @@
}
if (i < sysLoad.length - 1) {
netPayloadReadBWArray.push([ i, computeRate(netPayloadBytesRead, rrdPtr) ]);
+ netPayloadReadBWArray.push([ i, computeRate(datasetNetPayloadBytesRead, rrdPtr) ]);
netPayloadWriteBWArray.push([ i, computeRate(netPayloadBytesWritten, rrdPtr) ]);
+ netPayloadWriteBWArray.push([ i, computeRate(datasetNetPayloadBytesWritten, rrdPtr) ]);
netSignalingReadBWArray.push([ i, computeRate(netSignalingBytesRead, rrdPtr) ]);
+ netSignalingReadBWArray.push([ i, computeRate(datasetNetSignalingBytesRead, rrdPtr) ]);
netSignalingWriteBWArray.push([ i, computeRate(netSignalingBytesWritten, rrdPtr) ]);
+ netSignalingWriteBWArray.push([ i, computeRate(etSignalingBytesWritten, rrdPtr) ]);
ipcMessageSendRateArray.push([ i, computeRate(ipcMessagesSent, rrdPtr) ]);
ipcMessageBytesSendRateArray.push([ i, computeRate(ipcMessageBytesSent, rrdPtr) ]);
ipcMessageReceiveRateArray.push([ i, computeRate(ipcMessagesReceived, rrdPtr) ]);
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
index 0c5bb2f..55e4479 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/base/IClusterController.java
@@ -16,7 +16,9 @@
import java.util.List;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
import edu.uci.ics.hyracks.control.common.controllers.NodeRegistration;
@@ -46,6 +48,13 @@
public void registerPartitionRequest(PartitionRequest partitionRequest) throws Exception;
+ public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult, int partition,
+ int nPartitions, NetworkAddress networkAddress) throws Exception;
+
+ public void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) throws Exception;
+
+ public void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition) throws Exception;
+
public void notifyApplicationStateChange(String nodeId, String appName, ApplicationStatus status) throws Exception;
public void sendApplicationMessageToCC(byte[] data, String appName, String nodeId) throws Exception;
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
index 167eb4b..5071bc9 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NCConfig.java
@@ -37,6 +37,9 @@
@Option(name = "-data-ip-address", usage = "IP Address to bind data listener", required = true)
public String dataIPAddress;
+ @Option(name = "-result-ip-address", usage = "IP Address to bind dataset result distribution listener", required = true)
+ public String datasetIPAddress;
+
@Option(name = "-iodevices", usage = "Comma separated list of IO Device mount points (default: One device in default temp folder)", required = false)
public String ioDevices = System.getProperty("java.io.tmpdir");
@@ -55,6 +58,9 @@
@Option(name = "-max-memory", usage = "Maximum memory usable at this Node Controller in bytes (default: -1 auto)")
public int maxMemory = -1;
+ @Option(name = "-result-manager-memory", usage = "Memory usable for result caching at this Node Controller in bytes (default: -1 auto)")
+ public int resultManagerMemory = -1;
+
public void toCommandLine(List<String> cList) {
cList.add("-cc-host");
cList.add(ccHost);
@@ -66,6 +72,7 @@
cList.add(nodeId);
cList.add("-data-ip-address");
cList.add(dataIPAddress);
+ cList.add(datasetIPAddress);
cList.add("-iodevices");
cList.add(ioDevices);
cList.add("-dcache-client-servers");
@@ -80,5 +87,7 @@
cList.add(String.valueOf(nNetThreads));
cList.add("-max-memory");
cList.add(String.valueOf(maxMemory));
+ cList.add("-result-manager-memory");
+ cList.add(String.valueOf(resultManagerMemory));
}
}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
index 91cfecf..a897602 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/controllers/NodeRegistration.java
@@ -33,6 +33,8 @@
private final NetworkAddress dataPort;
+ private final NetworkAddress datasetPort;
+
private final String osName;
private final String arch;
@@ -60,13 +62,14 @@
private final HeartbeatSchema hbSchema;
public NodeRegistration(InetSocketAddress ncAddress, String nodeId, NCConfig ncConfig, NetworkAddress dataPort,
- String osName, String arch, String osVersion, int nProcessors, String vmName, String vmVersion,
- String vmVendor, String classpath, String libraryPath, String bootClasspath, List<String> inputArguments,
- Map<String, String> systemProperties, HeartbeatSchema hbSchema) {
+ NetworkAddress datasetPort, String osName, String arch, String osVersion, int nProcessors, String vmName,
+ String vmVersion, String vmVendor, String classpath, String libraryPath, String bootClasspath,
+ List<String> inputArguments, Map<String, String> systemProperties, HeartbeatSchema hbSchema) {
this.ncAddress = ncAddress;
this.nodeId = nodeId;
this.ncConfig = ncConfig;
this.dataPort = dataPort;
+ this.datasetPort = datasetPort;
this.osName = osName;
this.arch = arch;
this.osVersion = osVersion;
@@ -98,6 +101,10 @@
return dataPort;
}
+ public NetworkAddress getDatasetPort() {
+ return datasetPort;
+ }
+
public String getOSName() {
return osName;
}
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
index 1dba3bc..663c68a 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/heartbeat/HeartbeatData.java
@@ -37,6 +37,10 @@
public long netPayloadBytesWritten;
public long netSignalingBytesRead;
public long netSignalingBytesWritten;
+ public long datasetNetPayloadBytesRead;
+ public long datasetNetPayloadBytesWritten;
+ public long datasetNetSignalingBytesRead;
+ public long datasetNetSignalingBytesWritten;
public long ipcMessagesSent;
public long ipcMessageBytesSent;
public long ipcMessagesReceived;
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
index 557a8cb..b506b12 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/CCNCFunctions.java
@@ -36,6 +36,7 @@
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.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.job.JobFlag;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.job.JobStatus;
@@ -68,6 +69,9 @@
REPORT_PROFILE,
REGISTER_PARTITION_PROVIDER,
REGISTER_PARTITION_REQUEST,
+ REGISTER_RESULT_PARTITION_LOCATION,
+ REPORT_RESULT_PARTITION_WRITE_COMPLETION,
+ REPORT_RESULT_PARTITION_FAILURE,
APPLICATION_STATE_CHANGE_RESPONSE,
NODE_REGISTRATION_RESULT,
@@ -438,6 +442,127 @@
}
}
+ public static class RegisterResultPartitionLocationFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final boolean orderedResult;
+
+ private final int partition;
+
+ private final int nPartitions;
+
+ private NetworkAddress networkAddress;
+
+ public RegisterResultPartitionLocationFunction(JobId jobId, ResultSetId rsId, boolean orderedResult,
+ int partition, int nPartitions, NetworkAddress networkAddress) {
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.orderedResult = orderedResult;
+ this.partition = partition;
+ this.nPartitions = nPartitions;
+ this.networkAddress = networkAddress;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.REGISTER_RESULT_PARTITION_LOCATION;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public ResultSetId getResultSetId() {
+ return rsId;
+ }
+
+ public boolean getOrderedResult() {
+ return orderedResult;
+ }
+
+ public int getPartition() {
+ return partition;
+ }
+
+ public int getNPartitions() {
+ return nPartitions;
+ }
+
+ public NetworkAddress getNetworkAddress() {
+ return networkAddress;
+ }
+ }
+
+ public static class ReportResultPartitionWriteCompletionFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final int partition;
+
+ public ReportResultPartitionWriteCompletionFunction(JobId jobId, ResultSetId rsId, int partition) {
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.partition = partition;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.REPORT_RESULT_PARTITION_WRITE_COMPLETION;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public ResultSetId getResultSetId() {
+ return rsId;
+ }
+
+ public int getPartition() {
+ return partition;
+ }
+ }
+
+ public static class ReportResultPartitionFailureFunction extends Function {
+ private static final long serialVersionUID = 1L;
+
+ private final JobId jobId;
+
+ private final ResultSetId rsId;
+
+ private final int partition;
+
+ public ReportResultPartitionFailureFunction(JobId jobId, ResultSetId rsId, int partition) {
+ this.jobId = jobId;
+ this.rsId = rsId;
+ this.partition = partition;
+ }
+
+ @Override
+ public FunctionId getFunctionId() {
+ return FunctionId.REPORT_RESULT_PARTITION_FAILURE;
+ }
+
+ public JobId getJobId() {
+ return jobId;
+ }
+
+ public ResultSetId getResultSetId() {
+ return rsId;
+ }
+
+ public int getPartition() {
+ return partition;
+ }
+ }
+
public static class ApplicationStateChangeResponseFunction extends Function {
private static final long serialVersionUID = 1L;
diff --git a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
index bbaab4e..091a5d2 100644
--- a/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
+++ b/hyracks/hyracks-control/hyracks-control-common/src/main/java/edu/uci/ics/hyracks/control/common/ipc/ClusterControllerRemoteProxy.java
@@ -16,7 +16,9 @@
import java.util.List;
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.control.common.application.ApplicationStatus;
import edu.uci.ics.hyracks.control.common.base.IClusterController;
@@ -95,6 +97,28 @@
}
@Override
+ public void registerResultPartitionLocation(JobId jobId, ResultSetId rsId, boolean orderedResult, int partition,
+ int nPartitions, NetworkAddress networkAddress) throws Exception {
+ CCNCFunctions.RegisterResultPartitionLocationFunction fn = new CCNCFunctions.RegisterResultPartitionLocationFunction(
+ jobId, rsId, orderedResult, partition, nPartitions, networkAddress);
+ ipcHandle.send(-1, fn, null);
+ }
+
+ @Override
+ public void reportResultPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) throws Exception {
+ CCNCFunctions.ReportResultPartitionWriteCompletionFunction fn = new CCNCFunctions.ReportResultPartitionWriteCompletionFunction(
+ jobId, rsId, partition);
+ ipcHandle.send(-1, fn, null);
+ }
+
+ @Override
+ public void reportResultPartitionFailure(JobId jobId, ResultSetId rsId, int partition) throws Exception {
+ CCNCFunctions.ReportResultPartitionFailureFunction fn = new CCNCFunctions.ReportResultPartitionFailureFunction(
+ jobId, rsId, partition);
+ ipcHandle.send(-1, fn, null);
+ }
+
+ @Override
public void notifyApplicationStateChange(String nodeId, String appName, ApplicationStatus status) throws Exception {
CCNCFunctions.ApplicationStateChangeResponseFunction fn = new CCNCFunctions.ApplicationStateChangeResponseFunction(
nodeId, appName, status);
diff --git a/hyracks/hyracks-control/hyracks-control-nc/pom.xml b/hyracks/hyracks-control/hyracks-control-nc/pom.xml
index 6a7e5d8..c44cec9 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/pom.xml
+++ b/hyracks/hyracks-control/hyracks-control-nc/pom.xml
@@ -41,6 +41,11 @@
<artifactId>hyracks-net</artifactId>
<version>0.2.3-SNAPSHOT</version>
</dependency>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-comm</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ </dependency>
</dependencies>
<reporting>
<plugins>
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
index 0195143..8a36dac 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/NodeControllerService.java
@@ -45,6 +45,7 @@
import edu.uci.ics.hyracks.api.client.NodeControllerInfo;
import edu.uci.ics.hyracks.api.context.IHyracksRootContext;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
import edu.uci.ics.hyracks.api.io.IODeviceHandle;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.control.common.AbstractRemoteService;
@@ -61,7 +62,9 @@
import edu.uci.ics.hyracks.control.common.work.FutureValue;
import edu.uci.ics.hyracks.control.common.work.WorkQueue;
import edu.uci.ics.hyracks.control.nc.application.NCApplicationContext;
+import edu.uci.ics.hyracks.control.nc.dataset.DatasetPartitionManager;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.control.nc.net.DatasetNetworkManager;
import edu.uci.ics.hyracks.control.nc.net.NetworkManager;
import edu.uci.ics.hyracks.control.nc.partitions.PartitionManager;
import edu.uci.ics.hyracks.control.nc.runtime.RootHyracksContext;
@@ -94,6 +97,10 @@
private final NetworkManager netManager;
+ private final IDatasetPartitionManager datasetPartitionManager;
+
+ private final DatasetNetworkManager datasetNetworkManager;
+
private final WorkQueue queue;
private final Timer timer;
@@ -140,7 +147,11 @@
throw new Exception("id not set");
}
partitionManager = new PartitionManager(this);
- netManager = new NetworkManager(getIpAddress(ncConfig), partitionManager, ncConfig.nNetThreads);
+ netManager = new NetworkManager(getIpAddress(ncConfig.dataIPAddress), partitionManager, ncConfig.nNetThreads);
+
+ datasetPartitionManager = new DatasetPartitionManager(this, executor, ncConfig.resultManagerMemory);
+ datasetNetworkManager = new DatasetNetworkManager(getIpAddress(ncConfig.datasetIPAddress),
+ datasetPartitionManager, ncConfig.nNetThreads);
queue = new WorkQueue();
jobletMap = new Hashtable<JobId, Joblet>();
@@ -205,6 +216,7 @@
LOGGER.log(Level.INFO, "Starting NodeControllerService");
ipc.start();
netManager.start();
+ datasetNetworkManager.start();
IIPCHandle ccIPCHandle = ipc.getHandle(new InetSocketAddress(ncConfig.ccHost, ncConfig.ccPort));
this.ccs = new ClusterControllerRemoteProxy(ccIPCHandle);
HeartbeatSchema.GarbageCollectorInfo[] gcInfos = new HeartbeatSchema.GarbageCollectorInfo[gcMXBeans.size()];
@@ -213,10 +225,11 @@
}
HeartbeatSchema hbSchema = new HeartbeatSchema(gcInfos);
ccs.registerNode(new NodeRegistration(ipc.getSocketAddress(), id, ncConfig, netManager.getNetworkAddress(),
- osMXBean.getName(), osMXBean.getArch(), osMXBean.getVersion(), osMXBean.getAvailableProcessors(),
- runtimeMXBean.getVmName(), runtimeMXBean.getVmVersion(), runtimeMXBean.getVmVendor(), runtimeMXBean
- .getClassPath(), runtimeMXBean.getLibraryPath(), runtimeMXBean.getBootClassPath(),
- runtimeMXBean.getInputArguments(), runtimeMXBean.getSystemProperties(), hbSchema));
+ datasetNetworkManager.getNetworkAddress(), osMXBean.getName(), osMXBean.getArch(), osMXBean
+ .getVersion(), osMXBean.getAvailableProcessors(), runtimeMXBean.getVmName(), runtimeMXBean
+ .getVmVersion(), runtimeMXBean.getVmVendor(), runtimeMXBean.getClassPath(), runtimeMXBean
+ .getLibraryPath(), runtimeMXBean.getBootClassPath(), runtimeMXBean.getInputArguments(),
+ runtimeMXBean.getSystemProperties(), hbSchema));
synchronized (this) {
while (registrationPending) {
@@ -247,8 +260,10 @@
LOGGER.log(Level.INFO, "Stopping NodeControllerService");
executor.shutdownNow();
partitionManager.close();
+ datasetPartitionManager.close();
heartbeatTask.cancel();
netManager.stop();
+ datasetNetworkManager.stop();
queue.stop();
LOGGER.log(Level.INFO, "Stopped NodeControllerService");
}
@@ -273,6 +288,10 @@
return netManager;
}
+ public DatasetNetworkManager getDatasetNetworkManager() {
+ return datasetNetworkManager;
+ }
+
public PartitionManager getPartitionManager() {
return partitionManager;
}
@@ -297,8 +316,7 @@
return queue;
}
- private static InetAddress getIpAddress(NCConfig ncConfig) throws Exception {
- String ipaddrStr = ncConfig.dataIPAddress;
+ private static InetAddress getIpAddress(String ipaddrStr) throws Exception {
ipaddrStr = ipaddrStr.trim();
Pattern pattern = Pattern.compile("(\\d{1,3})\\.(\\d{1,3})\\.(\\d{1,3})\\.(\\d{1,3})");
Matcher m = pattern.matcher(ipaddrStr);
@@ -355,6 +373,12 @@
hbData.netSignalingBytesRead = netPC.getSignalingBytesRead();
hbData.netSignalingBytesWritten = netPC.getSignalingBytesWritten();
+ MuxDemuxPerformanceCounters datasetNetPC = datasetNetworkManager.getPerformanceCounters();
+ hbData.datasetNetPayloadBytesRead = datasetNetPC.getPayloadBytesRead();
+ hbData.datasetNetPayloadBytesWritten = datasetNetPC.getPayloadBytesWritten();
+ hbData.datasetNetSignalingBytesRead = datasetNetPC.getSignalingBytesRead();
+ hbData.datasetNetSignalingBytesWritten = datasetNetPC.getSignalingBytesWritten();
+
IPCPerformanceCounters ipcPC = ipc.getPerformanceCounters();
hbData.ipcMessagesSent = ipcPC.getMessageSentCount();
hbData.ipcMessageBytesSent = ipcPC.getMessageBytesSent();
@@ -459,6 +483,10 @@
}
}
+ public IDatasetPartitionManager getDatasetPartitionManager() {
+ return datasetPartitionManager;
+ }
+
public void sendApplicationMessageToCC(byte[] data, String appName, String nodeId) throws Exception {
ccs.sendApplicationMessageToCC(data, appName, nodeId);
}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
index eba3ec9..5a3e9dd 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/Task.java
@@ -34,6 +34,7 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.io.FileReference;
@@ -348,6 +349,11 @@
}
@Override
+ public IDatasetPartitionManager getDatasetPartitionManager() {
+ return ncs.getDatasetPartitionManager();
+ }
+
+ @Override
public void sendApplicationMessageToCC(byte[] message, String nodeId) throws Exception {
this.ncs.sendApplicationMessageToCC(message, this.getJobletContext().getApplicationContext()
.getApplicationName(), nodeId);
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetMemoryManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetMemoryManager.java
new file mode 100644
index 0000000..cecd677
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetMemoryManager.java
@@ -0,0 +1,237 @@
+/*
+ * 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.nc.dataset;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionWriter;
+import edu.uci.ics.hyracks.api.dataset.Page;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.partitions.ResultSetPartitionId;
+
+public class DatasetMemoryManager {
+ private final Set<Page> availPages;
+
+ private final LeastRecentlyUsedList leastRecentlyUsedList;
+
+ private final Map<ResultSetPartitionId, PartitionNode> resultPartitionNodesMap;
+
+ private final static int FRAME_SIZE = 32768;
+
+ public DatasetMemoryManager(int availableMemory) {
+ availPages = new HashSet<Page>();
+
+ // Atleast have one page for temporarily storing the results.
+ if (availableMemory <= 0)
+ availableMemory = FRAME_SIZE;
+
+ while (availableMemory >= FRAME_SIZE) {
+ /* TODO(madhusudancs): Should we have some way of accounting this memory usage by using Hyrack's allocateFrame()
+ * instead of direct ByteBuffer.allocate()?
+ */
+ availPages.add(new Page(ByteBuffer.allocate(FRAME_SIZE)));
+ availableMemory -= FRAME_SIZE;
+ }
+
+ leastRecentlyUsedList = new LeastRecentlyUsedList();
+ resultPartitionNodesMap = new HashMap<ResultSetPartitionId, PartitionNode>();
+ }
+
+ public Page requestPage(ResultSetPartitionId resultSetPartitionId, IDatasetPartitionWriter dpw)
+ throws OutOfMemoryError, HyracksDataException {
+ Page page;
+ if (availPages.isEmpty()) {
+ page = evictPage();
+ } else {
+ page = getAvailablePage();
+ }
+
+ page.clear();
+
+ /*
+ * It is extremely important to update the reference after obtaining the page because, in the cases where memory
+ * manager is allocated only one page of memory, the front of the LRU list should not be created by the
+ * update reference call before a page is pushed on to the element of the LRU list. So we first obtain the page,
+ * then make a updateReference call which in turn creates a new node in the LRU list and then add the page to it.
+ */
+ PartitionNode pn = updateReference(resultSetPartitionId, dpw);
+ pn.add(page);
+ return page;
+ }
+
+ public void pageReferenced(ResultSetPartitionId resultSetPartitionId) {
+ // When a page is referenced the dataset partition writer should already be known, so we pass null.
+ updateReference(resultSetPartitionId, null);
+ }
+
+ public int getPageSize() {
+ return FRAME_SIZE;
+ }
+
+ protected void insertPartitionNode(ResultSetPartitionId resultSetPartitionId, PartitionNode pn) {
+ leastRecentlyUsedList.add(pn);
+ resultPartitionNodesMap.put(resultSetPartitionId, pn);
+ }
+
+ protected synchronized PartitionNode updateReference(ResultSetPartitionId resultSetPartitionId,
+ IDatasetPartitionWriter dpw) {
+ PartitionNode pn = null;
+
+ if (!resultPartitionNodesMap.containsKey(resultSetPartitionId)) {
+ if (dpw != null) {
+ pn = new PartitionNode(resultSetPartitionId, dpw);
+ insertPartitionNode(resultSetPartitionId, pn);
+ }
+ return pn;
+ }
+ pn = resultPartitionNodesMap.get(resultSetPartitionId);
+ leastRecentlyUsedList.remove(pn);
+ insertPartitionNode(resultSetPartitionId, pn);
+
+ return pn;
+ }
+
+ protected synchronized Page evictPage() throws HyracksDataException {
+ PartitionNode pn = leastRecentlyUsedList.getFirst();
+ IDatasetPartitionWriter dpw = pn.getDatasetPartitionWriter();
+ Page page = dpw.returnPage();
+
+ /* If the partition holding the pages breaks the contract by not returning the page or it has no page, just take
+ * away all the pages allocated to it and add to the available pages set.
+ */
+ if (page == null) {
+ availPages.addAll(pn);
+ pn.clear();
+ resultPartitionNodesMap.remove(pn.getResultSetPartitionId());
+ leastRecentlyUsedList.remove(pn);
+
+ /* Based on the assumption that if the dataset partition writer returned a null page, it should be lying about
+ * the number of pages it holds in which case we just evict all the pages it holds and should thus be able to
+ * add all those pages to available set and we have at least one page to allocate back.
+ */
+ page = getAvailablePage();
+ } else {
+ pn.remove(page);
+
+ // If the partition no more holds any pages, remove it from the linked list and the hash map.
+ if (pn.isEmpty()) {
+ resultPartitionNodesMap.remove(pn.getResultSetPartitionId());
+ leastRecentlyUsedList.remove(pn);
+ }
+ }
+
+ return page;
+ }
+
+ protected synchronized Page getAvailablePage() {
+ Iterator<Page> iter = availPages.iterator();
+ Page page = iter.next();
+ iter.remove();
+ return page;
+ }
+
+ private class LeastRecentlyUsedList {
+ private PartitionNode head;
+
+ private PartitionNode tail;
+
+ public LeastRecentlyUsedList() {
+ head = null;
+ tail = null;
+ }
+
+ public void add(PartitionNode node) {
+ if (head == null) {
+ head = tail = node;
+ return;
+ }
+ tail.setNext(node);
+ node.setPrev(tail);
+ tail = node;
+ }
+
+ public void remove(PartitionNode node) {
+ if ((node == head) && (node == tail)) {
+ head = tail = null;
+ return;
+ } else if (node == head) {
+ head = head.getNext();
+ head.setPrev(null);
+ return;
+ } else if (node == tail) {
+ tail = tail.getPrev();
+ tail.setNext(null);
+ return;
+ } else {
+ PartitionNode prev = node.getPrev();
+ PartitionNode next = node.getNext();
+ prev.setNext(next);
+ next.setPrev(prev);
+ }
+ }
+
+ public PartitionNode getFirst() {
+ return head;
+ }
+ }
+
+ private class PartitionNode extends HashSet<Page> {
+ private static final long serialVersionUID = 1L;
+
+ private final ResultSetPartitionId resultSetPartitionId;
+
+ private final IDatasetPartitionWriter datasetPartitionWriter;
+
+ private PartitionNode prev;
+
+ private PartitionNode next;
+
+ public PartitionNode(ResultSetPartitionId resultSetPartitionId, IDatasetPartitionWriter datasetPartitionWriter) {
+ this.resultSetPartitionId = resultSetPartitionId;
+ this.datasetPartitionWriter = datasetPartitionWriter;
+ prev = null;
+ next = null;
+ }
+
+ public ResultSetPartitionId getResultSetPartitionId() {
+ return resultSetPartitionId;
+ }
+
+ public IDatasetPartitionWriter getDatasetPartitionWriter() {
+ return datasetPartitionWriter;
+ }
+
+ public void setPrev(PartitionNode node) {
+ prev = node;
+ }
+
+ public PartitionNode getPrev() {
+ return prev;
+ }
+
+ public void setNext(PartitionNode node) {
+ next = node;
+ }
+
+ public PartitionNode getNext() {
+ return next;
+ }
+ }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionManager.java
new file mode 100644
index 0000000..1cad54b
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionManager.java
@@ -0,0 +1,124 @@
+/*
+ * 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.nc.dataset;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Executor;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionReader;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.IWorkspaceFileFactory;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.control.nc.io.IOManager;
+import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
+import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
+
+public class DatasetPartitionManager implements IDatasetPartitionManager {
+ private final NodeControllerService ncs;
+
+ private final Executor executor;
+
+ private final Map<JobId, ResultState[]> partitionResultStateMap;
+
+ private final DefaultDeallocatableRegistry deallocatableRegistry;
+
+ private final IWorkspaceFileFactory fileFactory;
+
+ private final DatasetMemoryManager datasetMemoryManager;
+
+ public DatasetPartitionManager(NodeControllerService ncs, Executor executor, int availableMemory) {
+ this.ncs = ncs;
+ this.executor = executor;
+ partitionResultStateMap = new HashMap<JobId, ResultState[]>();
+ deallocatableRegistry = new DefaultDeallocatableRegistry();
+ fileFactory = new WorkspaceFileFactory(deallocatableRegistry, (IOManager) ncs.getRootContext().getIOManager());
+ datasetMemoryManager = new DatasetMemoryManager(availableMemory);
+ }
+
+ @Override
+ public IFrameWriter createDatasetPartitionWriter(IHyracksTaskContext ctx, ResultSetId rsId, boolean orderedResult,
+ int partition, int nPartitions) throws HyracksException {
+ DatasetPartitionWriter dpw = null;
+ JobId jobId = ctx.getJobletContext().getJobId();
+ try {
+ ncs.getClusterController().registerResultPartitionLocation(jobId, rsId, orderedResult, partition,
+ nPartitions, ncs.getDatasetNetworkManager().getNetworkAddress());
+ dpw = new DatasetPartitionWriter(ctx, this, jobId, rsId, partition, datasetMemoryManager);
+
+ ResultState[] resultStates = partitionResultStateMap.get(jobId);
+ if (resultStates == null) {
+ resultStates = new ResultState[nPartitions];
+ partitionResultStateMap.put(jobId, resultStates);
+ }
+ resultStates[partition] = dpw.getResultState();
+ } catch (Exception e) {
+ throw new HyracksException(e);
+ }
+
+ return dpw;
+ }
+
+ @Override
+ public void reportPartitionWriteCompletion(JobId jobId, ResultSetId rsId, int partition) throws HyracksException {
+ try {
+ ncs.getClusterController().reportResultPartitionWriteCompletion(jobId, rsId, partition);
+ } catch (Exception e) {
+ throw new HyracksException(e);
+ }
+ }
+
+ @Override
+ public void reportPartitionFailure(JobId jobId, ResultSetId rsId, int partition) throws HyracksException {
+ try {
+ ncs.getClusterController().reportResultPartitionFailure(jobId, rsId, partition);
+ } catch (Exception e) {
+ throw new HyracksException(e);
+ }
+ }
+
+ @Override
+ public void initializeDatasetPartitionReader(JobId jobId, int partition, IFrameWriter writer)
+ throws HyracksException {
+ ResultState[] resultStates = partitionResultStateMap.get(jobId);
+
+ if (resultStates == null) {
+ throw new HyracksException("Unknown JobId " + jobId);
+ }
+
+ ResultState resultState = resultStates[partition];
+ if (resultState == null) {
+ throw new HyracksException("No DatasetPartitionWriter for partition " + partition);
+ }
+
+ IDatasetPartitionReader dpr = new DatasetPartitionReader(datasetMemoryManager, executor, resultState);
+ dpr.writeTo(writer);
+ }
+
+ @Override
+ public IWorkspaceFileFactory getFileFactory() {
+ return fileFactory;
+ }
+
+ @Override
+ public void close() {
+ deallocatableRegistry.close();
+ }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionReader.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionReader.java
new file mode 100644
index 0000000..296c502
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionReader.java
@@ -0,0 +1,122 @@
+/*
+ * 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.nc.dataset;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.Executor;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionReader;
+import edu.uci.ics.hyracks.api.dataset.Page;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.IFileHandle;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.comm.channels.NetworkOutputChannel;
+
+public class DatasetPartitionReader implements IDatasetPartitionReader {
+ private static final Logger LOGGER = Logger.getLogger(DatasetPartitionReader.class.getName());
+
+ private final DatasetMemoryManager datasetMemoryManager;
+
+ private final Executor executor;
+
+ private final ResultState resultState;
+
+ private IFileHandle fileHandle;
+
+ public DatasetPartitionReader(DatasetMemoryManager datasetMemoryManager, Executor executor, ResultState resultState) {
+ this.datasetMemoryManager = datasetMemoryManager;
+ this.executor = executor;
+ this.resultState = resultState;
+ }
+
+ private long read(long offset, ByteBuffer buffer) throws HyracksDataException {
+ long readSize = 0;
+ synchronized (resultState) {
+ while (offset >= resultState.getSize() && !resultState.getEOS()) {
+ try {
+ resultState.wait();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ }
+
+ if (offset >= resultState.getSize() && resultState.getEOS()) {
+ return readSize;
+ }
+
+ if (offset < resultState.getPersistentSize()) {
+ readSize = resultState.getIOManager().syncRead(fileHandle, offset, buffer);
+ }
+
+ if (readSize < buffer.capacity()) {
+ long localPageOffset = offset - resultState.getPersistentSize();
+ int localPageIndex = (int) (localPageOffset / datasetMemoryManager.getPageSize());
+ int pageOffset = (int) (localPageOffset % datasetMemoryManager.getPageSize());
+ Page page = resultState.getPage(localPageIndex);
+ readSize += buffer.remaining();
+ buffer.put(page.getBuffer().array(), pageOffset, buffer.remaining());
+ }
+
+ datasetMemoryManager.pageReferenced(resultState.getResultSetPartitionId());
+ return readSize;
+ }
+
+ @Override
+ public void writeTo(final IFrameWriter writer) {
+ executor.execute(new Runnable() {
+ @Override
+ public void run() {
+ NetworkOutputChannel channel = (NetworkOutputChannel) writer;
+ channel.setFrameSize(resultState.getFrameSize());
+ try {
+ fileHandle = resultState.getIOManager().open(resultState.getValidFileReference(),
+ IIOManager.FileReadWriteMode.READ_ONLY, IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+ channel.open();
+ try {
+ long offset = 0;
+ ByteBuffer buffer = ByteBuffer.allocate(resultState.getFrameSize());
+ while (true) {
+ buffer.clear();
+ long size = read(offset, buffer);
+ if (size <= 0) {
+ break;
+ } else if (size < buffer.limit()) {
+ throw new HyracksDataException("Premature end of file - readSize: " + size
+ + " buffer limit: " + buffer.limit());
+ }
+ offset += size;
+ buffer.flip();
+ channel.nextFrame(buffer);
+ }
+ } finally {
+ channel.close();
+ resultState.getIOManager().close(fileHandle);
+ }
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ } catch (HyracksDataException e) {
+ throw new RuntimeException(e);
+ }
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("result reading successful(" + resultState.getResultSetPartitionId() + ")");
+ }
+ }
+ });
+ }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
new file mode 100644
index 0000000..f6ae540
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/DatasetPartitionWriter.java
@@ -0,0 +1,148 @@
+/*
+ * 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.nc.dataset;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionWriter;
+import edu.uci.ics.hyracks.api.dataset.Page;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IFileHandle;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.ResultSetPartitionId;
+
+public class DatasetPartitionWriter implements IDatasetPartitionWriter {
+ private static final Logger LOGGER = Logger.getLogger(DatasetPartitionWriter.class.getName());
+
+ private static final String FILE_PREFIX = "result_";
+
+ private final IDatasetPartitionManager manager;
+
+ private final JobId jobId;
+
+ private final ResultSetId resultSetId;
+
+ private final int partition;
+
+ private final DatasetMemoryManager datasetMemoryManager;
+
+ private final ResultSetPartitionId resultSetPartitionId;
+
+ private final ResultState resultState;
+
+ private IFileHandle fileHandle;
+
+ public DatasetPartitionWriter(IHyracksTaskContext ctx, IDatasetPartitionManager manager, JobId jobId,
+ ResultSetId rsId, int partition, DatasetMemoryManager datasetMemoryManager) {
+ this.manager = manager;
+ this.jobId = jobId;
+ this.resultSetId = rsId;
+ this.partition = partition;
+ this.datasetMemoryManager = datasetMemoryManager;
+
+ resultSetPartitionId = new ResultSetPartitionId(jobId, rsId, partition);
+ resultState = new ResultState(resultSetPartitionId, ctx.getIOManager(), ctx.getFrameSize());
+ }
+
+ public ResultState getResultState() {
+ return resultState;
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("open(" + partition + ")");
+ }
+ String fName = FILE_PREFIX + String.valueOf(partition);
+ FileReference fRef = manager.getFileFactory().createUnmanagedWorkspaceFile(fName);
+ fileHandle = resultState.getIOManager().open(fRef, IIOManager.FileReadWriteMode.READ_WRITE,
+ IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
+ resultState.init(fRef);
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ int srcOffset = 0;
+ Page destPage = resultState.getLastPage();
+
+ while (srcOffset < buffer.limit()) {
+ if ((destPage == null) || (destPage.getBuffer().remaining() <= 0)) {
+ destPage = datasetMemoryManager.requestPage(resultSetPartitionId, this);
+ resultState.addPage(destPage);
+ }
+ int srcLength = Math.min(buffer.limit() - srcOffset, destPage.getBuffer().remaining());
+ destPage.getBuffer().put(buffer.array(), srcOffset, srcLength);
+ srcOffset += srcLength;
+ resultState.incrementSize(srcLength);
+ }
+
+ synchronized (resultState) {
+ resultState.notifyAll();
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ try {
+ manager.reportPartitionFailure(jobId, resultSetId, partition);
+ } catch (HyracksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ if (LOGGER.isLoggable(Level.INFO)) {
+ LOGGER.info("close(" + partition + ")");
+ }
+
+ try {
+ synchronized (resultState) {
+ resultState.setEOS(true);
+ resultState.notifyAll();
+ }
+ manager.reportPartitionWriteCompletion(jobId, resultSetId, partition);
+ } catch (HyracksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public Page returnPage() throws HyracksDataException {
+ Page page = resultState.removePage(0);
+
+ IIOManager ioManager = resultState.getIOManager();
+
+ // If we do not have any pages to be given back close the write channel since we don't write any more, return null.
+ if (page == null) {
+ ioManager.close(fileHandle);
+ return null;
+ }
+
+ page.getBuffer().flip();
+
+ long delta = ioManager.syncWrite(fileHandle, resultState.getPersistentSize(), page.getBuffer());
+ resultState.incrementPersistentSize(delta);
+ return page;
+ }
+}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java
new file mode 100644
index 0000000..3db3fd9
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/dataset/ResultState.java
@@ -0,0 +1,173 @@
+/*
+ * 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.nc.dataset;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
+import edu.uci.ics.hyracks.api.dataset.Page;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.api.partitions.ResultSetPartitionId;
+
+public class ResultState implements IStateObject {
+ private final ResultSetPartitionId resultSetPartitionId;
+
+ private final int frameSize;
+
+ private final IIOManager ioManager;
+
+ private final AtomicBoolean eos;
+
+ private final AtomicBoolean readEOS;
+
+ private final List<Page> localPageList;
+
+ private FileReference fileRef;
+
+ private long size;
+
+ private long persistentSize;
+
+ ResultState(ResultSetPartitionId resultSetPartitionId, IIOManager ioManager, int frameSize) {
+ this.resultSetPartitionId = resultSetPartitionId;
+ this.ioManager = ioManager;
+ this.frameSize = frameSize;
+ eos = new AtomicBoolean(false);
+ readEOS = new AtomicBoolean(false);
+ localPageList = new ArrayList<Page>();
+ }
+
+ public synchronized void init(FileReference fileRef) {
+ this.fileRef = fileRef;
+
+ size = 0;
+ persistentSize = 0;
+ notifyAll();
+ }
+
+ public ResultSetPartitionId getResultSetPartitionId() {
+ return resultSetPartitionId;
+ }
+
+ public int getFrameSize() {
+ return frameSize;
+ }
+
+ public IIOManager getIOManager() {
+ return ioManager;
+ }
+
+ public synchronized void incrementSize(long delta) {
+ size += delta;
+ }
+
+ public synchronized long getSize() {
+ return size;
+ }
+
+ public synchronized void incrementPersistentSize(long delta) {
+ persistentSize += delta;
+ }
+
+ public synchronized long getPersistentSize() {
+ return persistentSize;
+ }
+
+ public void setEOS(boolean eos) {
+ this.eos.set(eos);
+ }
+
+ public boolean getEOS() {
+ return eos.get();
+ }
+
+ public boolean getReadEOS() {
+ return readEOS.get();
+ }
+
+ public synchronized void addPage(Page page) {
+ localPageList.add(page);
+ }
+
+ public synchronized Page removePage(int index) {
+ Page page = null;
+ if (!localPageList.isEmpty()) {
+ page = localPageList.remove(index);
+ }
+ return page;
+ }
+
+ public synchronized Page getPage(int index) {
+ Page page = null;
+ if (!localPageList.isEmpty()) {
+ page = localPageList.get(index);
+ }
+ return page;
+ }
+
+ public synchronized Page getLastPage() {
+ Page page = null;
+ if (!localPageList.isEmpty()) {
+ page = localPageList.get(localPageList.size() - 1);
+ }
+ return page;
+ }
+
+ public synchronized Page getFirstPage() {
+ Page page = null;
+ if (!localPageList.isEmpty()) {
+ page = localPageList.get(0);
+ }
+ return page;
+ }
+
+ public synchronized FileReference getValidFileReference() throws InterruptedException {
+ while (fileRef == null)
+ wait();
+ return fileRef;
+ }
+
+ @Override
+ public JobId getJobId() {
+ return resultSetPartitionId.getJobId();
+ }
+
+ @Override
+ public Object getId() {
+ return resultSetPartitionId;
+ }
+
+ @Override
+ public long getMemoryOccupancy() {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void toBytes(DataOutput out) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public void fromBytes(DataInput in) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java
new file mode 100644
index 0000000..5b8b333
--- /dev/null
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/DatasetNetworkManager.java
@@ -0,0 +1,127 @@
+/*
+ * 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.nc.net;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.api.comm.NetworkAddress;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.JobId;
+import edu.uci.ics.hyracks.comm.channels.IChannelConnectionFactory;
+import edu.uci.ics.hyracks.comm.channels.NetworkOutputChannel;
+import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
+import edu.uci.ics.hyracks.net.exceptions.NetException;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.ChannelControlBlock;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.IChannelOpenListener;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MultiplexedConnection;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemux;
+import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemuxPerformanceCounters;
+
+public class DatasetNetworkManager implements IChannelConnectionFactory {
+ private static final Logger LOGGER = Logger.getLogger(DatasetNetworkManager.class.getName());
+
+ private static final int MAX_CONNECTION_ATTEMPTS = 5;
+
+ static final int INITIAL_MESSAGE_SIZE = 20;
+
+ private final IDatasetPartitionManager partitionManager;
+
+ private final MuxDemux md;
+
+ private NetworkAddress networkAddress;
+
+ public DatasetNetworkManager(InetAddress inetAddress, IDatasetPartitionManager partitionManager, int nThreads)
+ throws IOException {
+ this.partitionManager = partitionManager;
+ md = new MuxDemux(new InetSocketAddress(inetAddress, 0), new ChannelOpenListener(), nThreads,
+ MAX_CONNECTION_ATTEMPTS);
+ }
+
+ public void start() throws IOException {
+ md.start();
+ InetSocketAddress sockAddr = md.getLocalAddress();
+ networkAddress = new NetworkAddress(sockAddr.getAddress().getAddress(), sockAddr.getPort());
+ }
+
+ public NetworkAddress getNetworkAddress() {
+ return networkAddress;
+ }
+
+ public void stop() {
+
+ }
+
+ public ChannelControlBlock connect(SocketAddress remoteAddress) throws InterruptedException, NetException {
+ MultiplexedConnection mConn = md.connect((InetSocketAddress) remoteAddress);
+ return mConn.openChannel();
+ }
+
+ private class ChannelOpenListener implements IChannelOpenListener {
+ @Override
+ public void channelOpened(ChannelControlBlock channel) {
+ channel.getReadInterface().setFullBufferAcceptor(new InitialBufferAcceptor(channel));
+ channel.getReadInterface().getEmptyBufferAcceptor().accept(ByteBuffer.allocate(INITIAL_MESSAGE_SIZE));
+ }
+ }
+
+ private class InitialBufferAcceptor implements ICloseableBufferAcceptor {
+ private final ChannelControlBlock ccb;
+
+ private NetworkOutputChannel noc;
+
+ public InitialBufferAcceptor(ChannelControlBlock ccb) {
+ this.ccb = ccb;
+ }
+
+ @Override
+ public void accept(ByteBuffer buffer) {
+ JobId jobId = new JobId(buffer.getLong());
+ int partition = buffer.getInt();
+ if (LOGGER.isLoggable(Level.FINE)) {
+ LOGGER.fine("Received initial dataset partition read request for JobId: " + jobId + " partition: "
+ + partition + " on channel: " + ccb);
+ }
+ noc = new NetworkOutputChannel(ccb, 1);
+ try {
+ partitionManager.initializeDatasetPartitionReader(jobId, partition, noc);
+ } catch (HyracksException e) {
+ noc.abort();
+ }
+ }
+
+ @Override
+ public void close() {
+
+ }
+
+ @Override
+ public void error(int ecode) {
+ if (noc != null) {
+ noc.abort();
+ }
+ }
+ }
+
+ public MuxDemuxPerformanceCounters getPerformanceCounters() {
+ return md.getPerformanceCounters();
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
index b805595..c8e4e94 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/net/NetworkManager.java
@@ -27,6 +27,8 @@
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.comm.channels.IChannelConnectionFactory;
+import edu.uci.ics.hyracks.comm.channels.NetworkOutputChannel;
import edu.uci.ics.hyracks.control.nc.partitions.PartitionManager;
import edu.uci.ics.hyracks.net.buffers.ICloseableBufferAcceptor;
import edu.uci.ics.hyracks.net.exceptions.NetException;
@@ -36,7 +38,7 @@
import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemux;
import edu.uci.ics.hyracks.net.protocols.muxdemux.MuxDemuxPerformanceCounters;
-public class NetworkManager {
+public class NetworkManager implements IChannelConnectionFactory {
private static final Logger LOGGER = Logger.getLogger(NetworkManager.class.getName());
private static final int MAX_CONNECTION_ATTEMPTS = 5;
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
index 16e31f7..ba6e6c3 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/MaterializedPartitionInputChannel.java
@@ -21,7 +21,7 @@
import edu.uci.ics.hyracks.api.channels.IInputChannel;
import edu.uci.ics.hyracks.api.channels.IInputChannelMonitor;
import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.context.IHyracksCommonContext;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.partitions.IPartition;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
@@ -82,7 +82,7 @@
}
@Override
- public void open(IHyracksTaskContext ctx) throws HyracksDataException {
+ public void open(IHyracksCommonContext ctx) throws HyracksDataException {
for (int i = 0; i < nBuffers; ++i) {
emptyQueue.add(ctx.allocateFrame());
}
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
index 45c091a..ea88a75 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/partitions/PartitionManager.java
@@ -28,12 +28,12 @@
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.partitions.IPartition;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.comm.channels.NetworkOutputChannel;
import edu.uci.ics.hyracks.control.common.job.PartitionDescriptor;
import edu.uci.ics.hyracks.control.common.job.PartitionState;
import edu.uci.ics.hyracks.control.nc.NodeControllerService;
import edu.uci.ics.hyracks.control.nc.io.IOManager;
import edu.uci.ics.hyracks.control.nc.io.WorkspaceFileFactory;
-import edu.uci.ics.hyracks.control.nc.net.NetworkOutputChannel;
import edu.uci.ics.hyracks.control.nc.resources.DefaultDeallocatableRegistry;
public class PartitionManager {
@@ -98,7 +98,7 @@
List<IPartition> pList = partitionMap.get(partitionId);
if (pList != null && !pList.isEmpty()) {
IPartition partition = pList.get(0);
- writer.setTaskContext(partition.getTaskContext());
+ writer.setFrameSize(partition.getTaskContext().getFrameSize());
partition.writeTo(writer);
if (!partition.isReusable()) {
partitionMap.remove(partitionId);
diff --git a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
index bb9669d..7ed9d11 100644
--- a/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
+++ b/hyracks/hyracks-control/hyracks-control-nc/src/main/java/edu/uci/ics/hyracks/control/nc/work/ReportPartitionAvailabilityWork.java
@@ -22,10 +22,10 @@
import edu.uci.ics.hyracks.api.comm.PartitionChannel;
import edu.uci.ics.hyracks.api.job.JobId;
import edu.uci.ics.hyracks.api.partitions.PartitionId;
+import edu.uci.ics.hyracks.comm.channels.NetworkInputChannel;
import edu.uci.ics.hyracks.control.common.work.AbstractWork;
import edu.uci.ics.hyracks.control.nc.Joblet;
import edu.uci.ics.hyracks.control.nc.NodeControllerService;
-import edu.uci.ics.hyracks.control.nc.net.NetworkInputChannel;
public class ReportPartitionAvailabilityWork extends AbstractWork {
private final NodeControllerService ncs;
diff --git a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java
index 8508287..c0ba163 100644
--- a/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java
+++ b/hyracks/hyracks-data/hyracks-data-std/src/main/java/edu/uci/ics/hyracks/data/std/util/ByteArrayAccessibleOutputStream.java
@@ -21,6 +21,14 @@
public class ByteArrayAccessibleOutputStream extends ByteArrayOutputStream {
private static final Logger LOGGER = Logger.getLogger(ByteArrayAccessibleOutputStream.class.getName());
+ public ByteArrayAccessibleOutputStream() {
+ super();
+ }
+
+ public ByteArrayAccessibleOutputStream(int size) {
+ super(size);
+ }
+
public byte[] getByteArray() {
return buf;
}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
new file mode 100644
index 0000000..07f6ba2
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/FrameOutputStream.java
@@ -0,0 +1,54 @@
+/*
+ * 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.dataflow.common.comm.io;
+
+import java.nio.ByteBuffer;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import edu.uci.ics.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+
+public class FrameOutputStream extends ByteArrayAccessibleOutputStream {
+ private static final Logger LOGGER = Logger.getLogger(FrameOutputStream.class.getName());
+
+ private final FrameTupleAppender frameTupleAppender;
+
+ public FrameOutputStream(int frameSize) {
+ super(frameSize);
+ this.frameTupleAppender = new FrameTupleAppender(frameSize);
+ }
+
+ public void reset(ByteBuffer buffer, boolean clear) {
+ frameTupleAppender.reset(buffer, clear);
+ }
+
+ public int getTupleCount() {
+ int tupleCount = frameTupleAppender.getTupleCount();
+ if (LOGGER.isLoggable(Level.FINEST)) {
+ LOGGER.finest("appendTuple(): tuple count: " + tupleCount);
+ }
+ return tupleCount;
+ }
+
+ public boolean appendTuple() {
+ if (LOGGER.isLoggable(Level.FINEST)) {
+ LOGGER.finest("appendTuple(): tuple size: " + count);
+ }
+ boolean appended = frameTupleAppender.append(buf, 0, count);
+ count = 0;
+ return appended;
+ }
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java
new file mode 100644
index 0000000..915a436
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/edu/uci/ics/hyracks/dataflow/common/comm/io/ResultFrameTupleAccessor.java
@@ -0,0 +1,97 @@
+/*
+ * 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.dataflow.common.comm.io;
+
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.FrameHelper;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+
+public class ResultFrameTupleAccessor implements IFrameTupleAccessor {
+
+ private final int frameSize;
+ private ByteBuffer buffer;
+
+ public ResultFrameTupleAccessor(int frameSize) {
+ this.frameSize = frameSize;
+ }
+
+ @Override
+ public void reset(ByteBuffer buffer) {
+ this.buffer = buffer;
+ }
+
+ @Override
+ public ByteBuffer getBuffer() {
+ return buffer;
+ }
+
+ @Override
+ public int getTupleCount() {
+ return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize));
+ }
+
+ @Override
+ public int getTupleStartOffset(int tupleIndex) {
+ return tupleIndex == 0 ? 0 : buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * tupleIndex);
+ }
+
+ @Override
+ public int getTupleEndOffset(int tupleIndex) {
+ return buffer.getInt(FrameHelper.getTupleCountOffset(frameSize) - 4 * (tupleIndex + 1));
+ }
+
+ @Override
+ public int getFieldStartOffset(int tupleIndex, int fIdx) {
+ return fIdx == 0 ? 0 : buffer.getInt(getTupleStartOffset(tupleIndex) + (fIdx - 1) * 4);
+ }
+
+ @Override
+ public int getFieldEndOffset(int tupleIndex, int fIdx) {
+ return buffer.getInt(getTupleStartOffset(tupleIndex) + fIdx * 4);
+ }
+
+ @Override
+ public int getFieldLength(int tupleIndex, int fIdx) {
+ return getFieldEndOffset(tupleIndex, fIdx) - getFieldStartOffset(tupleIndex, fIdx);
+ }
+
+ @Override
+ public int getFieldSlotsLength() {
+ return getFieldCount() * 4;
+ }
+
+ public void prettyPrint() {
+ ByteBufferInputStream bbis = new ByteBufferInputStream();
+ DataInputStream dis = new DataInputStream(bbis);
+ int tc = getTupleCount();
+ System.err.println("TC: " + tc);
+ for (int i = 0; i < tc; ++i) {
+ System.err.print(i + ":(" + getTupleStartOffset(i) + ", " + getTupleEndOffset(i) + ")[");
+
+ bbis.setByteBuffer(buffer, getTupleStartOffset(i));
+ System.err.print(dis);
+
+ System.err.println("]");
+ }
+ }
+
+ @Override
+ public int getFieldCount() {
+ return 1;
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
new file mode 100644
index 0000000..edca60a
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/edu/uci/ics/hyracks/dataflow/std/result/ResultWriterOperatorDescriptor.java
@@ -0,0 +1,120 @@
+/*
+ * 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.dataflow.std.result;
+
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.ByteBuffer;
+
+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.IResultSerializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameOutputStream;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+
+public class ResultWriterOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ private final ResultSetId rsId;
+
+ private final boolean ordered;
+
+ private final IResultSerializerFactory resultSerializerFactory;
+
+ public ResultWriterOperatorDescriptor(IOperatorDescriptorRegistry spec, ResultSetId rsId, boolean ordered,
+ IResultSerializerFactory resultSerializerFactory) throws IOException {
+ super(spec, 1, 0);
+ this.rsId = rsId;
+ this.ordered = ordered;
+ this.resultSerializerFactory = resultSerializerFactory;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) {
+ final IDatasetPartitionManager dpm = ctx.getDatasetPartitionManager();
+
+ final ByteBuffer outputBuffer = ctx.allocateFrame();
+
+ final FrameOutputStream frameOutputStream = new FrameOutputStream(ctx.getFrameSize());
+ frameOutputStream.reset(outputBuffer, true);
+ PrintStream printStream = new PrintStream(frameOutputStream);
+
+ final RecordDescriptor outRecordDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+ final IResultSerializer resultSerializer = resultSerializerFactory.createResultSerializer(outRecordDesc,
+ printStream);
+
+ final FrameTupleAccessor frameTupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(), outRecordDesc);
+
+ return new AbstractUnaryInputSinkOperatorNodePushable() {
+ IFrameWriter datasetPartitionWriter;
+
+ @Override
+ public void open() throws HyracksDataException {
+ try {
+ datasetPartitionWriter = dpm.createDatasetPartitionWriter(ctx, rsId, ordered, partition,
+ nPartitions);
+ datasetPartitionWriter.open();
+ resultSerializer.init();
+ } catch (HyracksException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ frameTupleAccessor.reset(buffer);
+ for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
+ resultSerializer.appendTuple(frameTupleAccessor, tIndex);
+ if (!frameOutputStream.appendTuple()) {
+ datasetPartitionWriter.nextFrame(outputBuffer);
+ frameOutputStream.reset(outputBuffer, true);
+
+ /* TODO(madhusudancs): This works under the assumption that no single serialized record is
+ * longer than the buffer size.
+ */
+ resultSerializer.appendTuple(frameTupleAccessor, tIndex);
+ frameOutputStream.appendTuple();
+ }
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ datasetPartitionWriter.fail();
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ if (frameOutputStream.getTupleCount() > 0) {
+ datasetPartitionWriter.nextFrame(outputBuffer);
+ frameOutputStream.reset(outputBuffer, true);
+ }
+ datasetPartitionWriter.close();
+ }
+ };
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml b/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
index 1daf1f7..5e7b5c9 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
@@ -32,6 +32,13 @@
<scope>test</scope>
</dependency>
<dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-client</artifactId>
+ <version>0.2.3-SNAPSHOT</version>
+ <type>jar</type>
+ <scope>compile</scope>
+ </dependency>
+ <dependency>
<groupId>edu.uci.ics.hyracks</groupId>
<artifactId>hyracks-dataflow-std</artifactId>
<version>0.2.3-SNAPSHOT</version>
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
index 8482083..33ddca2 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexScanOperatorTest.java
@@ -30,6 +30,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -44,8 +45,8 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -60,6 +61,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class BTreePrimaryIndexScanOperatorTest extends AbstractIntegrationTest {
static {
@@ -114,7 +116,7 @@
spec.addRoot(primaryCreateOp);
runTest(spec);
}
-
+
public void loadPrimaryIndexTest() throws Exception {
JobSpecification spec = new JobSpecification();
@@ -143,8 +145,9 @@
int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -179,13 +182,16 @@
int[] highKeyFields = null; // + infinity
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
- storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, lowKeyFields,
- highKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
index 82fecbe..acd3027 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexSearchOperatorTest.java
@@ -30,6 +30,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -44,8 +45,8 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -60,6 +61,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class BTreePrimaryIndexSearchOperatorTest extends AbstractIntegrationTest {
static {
@@ -114,7 +116,7 @@
spec.addRoot(primaryCreateOp);
runTest(spec);
}
-
+
public void loadPrimaryIndexTest() throws Exception {
JobSpecification spec = new JobSpecification();
@@ -143,8 +145,9 @@
int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -184,13 +187,16 @@
int[] highKeyFields = { 1 };
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
- storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, lowKeyFields,
- highKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
index e63ce11..ca03b16 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreePrimaryIndexStatsOperatorTest.java
@@ -29,6 +29,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -42,7 +43,7 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
@@ -57,6 +58,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class BTreePrimaryIndexStatsOperatorTest extends AbstractIntegrationTest {
static {
@@ -82,7 +84,6 @@
private IFileSplitProvider primaryBtreeSplitProvider = new ConstantFileSplitProvider(
new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
-
@Before
public void setup() throws Exception {
// field, type and key declarations for primary index
@@ -107,7 +108,7 @@
spec.addRoot(primaryCreateOp);
runTest(spec);
}
-
+
public void loadPrimaryIndexTest() throws Exception {
JobSpecification spec = new JobSpecification();
@@ -136,8 +137,9 @@
int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -153,12 +155,15 @@
JobSpecification spec = new JobSpecification();
TreeIndexStatsOperatorDescriptor primaryStatsOp = new TreeIndexStatsOperatorDescriptor(spec, storageManager,
- indexRegistryProvider, primaryBtreeSplitProvider,
- primaryTypeTraits, primaryComparatorFactories, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories,
+ dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryStatsOp, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), primaryStatsOp, 0, printer, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
index 3c87ae3..34a1cd3 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexInsertOperatorTest.java
@@ -30,6 +30,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -44,9 +45,9 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -63,6 +64,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class BTreeSecondaryIndexInsertOperatorTest extends AbstractIntegrationTest {
static {
@@ -339,9 +341,11 @@
primaryHighKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryBtreeSearchOp, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
index 1304f12..c9ee118 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/btree/BTreeSecondaryIndexSearchOperatorTest.java
@@ -30,6 +30,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -44,8 +45,8 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -60,6 +61,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class BTreeSecondaryIndexSearchOperatorTest extends AbstractIntegrationTest {
static {
@@ -137,7 +139,7 @@
spec.addRoot(primaryCreateOp);
runTest(spec);
}
-
+
public void loadPrimaryIndexTest() throws Exception {
JobSpecification spec = new JobSpecification();
@@ -166,8 +168,9 @@
int[] fieldPermutation = { 0, 1, 2, 4, 5, 7 };
TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -180,14 +183,14 @@
public void createSecondaryIndex() throws Exception {
JobSpecification spec = new JobSpecification();
- TreeIndexCreateOperatorDescriptor secondaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
- indexRegistryProvider, secondaryBtreeSplitProvider, secondaryTypeTraits, secondaryComparatorFactories,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ TreeIndexCreateOperatorDescriptor secondaryCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
+ storageManager, indexRegistryProvider, secondaryBtreeSplitProvider, secondaryTypeTraits,
+ secondaryComparatorFactories, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryCreateOp, NC1_ID);
spec.addRoot(secondaryCreateOp);
runTest(spec);
}
-
+
public void loadSecondaryIndexTest() throws Exception {
JobSpecification spec = new JobSpecification();
@@ -212,8 +215,9 @@
// scan primary index
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
- storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, lowKeyFields,
- highKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, dataflowHelperFactory, false,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
// sort based on secondary keys
@@ -225,8 +229,9 @@
// load secondary index
int[] fieldPermutation = { 3, 0 };
TreeIndexBulkLoadOperatorDescriptor secondaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- storageManager, indexRegistryProvider, secondaryBtreeSplitProvider, secondaryTypeTraits, secondaryComparatorFactories, fieldPermutation, 0.7f,
- dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, secondaryBtreeSplitProvider, secondaryTypeTraits,
+ secondaryComparatorFactories, fieldPermutation, 0.7f, dataflowHelperFactory,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeBulkLoad, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryBtreeSearchOp, 0);
@@ -268,8 +273,8 @@
// search secondary index
BTreeSearchOperatorDescriptor secondaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec,
secondaryRecDesc, storageManager, indexRegistryProvider, secondaryBtreeSplitProvider,
- secondaryTypeTraits, secondaryComparatorFactories, secondaryLowKeyFields, secondaryHighKeyFields, true, true,
- dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+ secondaryTypeTraits, secondaryComparatorFactories, secondaryLowKeyFields, secondaryHighKeyFields, true,
+ true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryBtreeSearchOp, NC1_ID);
int[] primaryLowKeyFields = { 1 }; // second field from the tuples
@@ -279,13 +284,16 @@
// search primary index
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
- storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits, primaryComparatorFactories, primaryLowKeyFields,
- primaryHighKeyFields, true, true, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryBtreeSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, primaryLowKeyFields, primaryHighKeyFields, true, true,
+ dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryBtreeSearchOp, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
index 023bdd9..8893567 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractIntegrationTest.java
@@ -14,29 +14,39 @@
*/
package edu.uci.ics.hyracks.tests.integration;
+import java.io.BufferedReader;
import java.io.File;
+import java.io.FileReader;
import java.io.IOException;
+import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.EnumSet;
import java.util.List;
import java.util.logging.Level;
import java.util.logging.Logger;
-import org.apache.commons.io.FileUtils;
import org.junit.AfterClass;
+import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.rules.TemporaryFolder;
import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
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.client.dataset.HyracksDataset;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ResultFrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
public abstract class AbstractIntegrationTest {
private static final Logger LOGGER = Logger.getLogger(AbstractIntegrationTest.class.getName());
@@ -80,6 +90,7 @@
ncConfig1.ccPort = 39001;
ncConfig1.clusterNetIPAddress = "127.0.0.1";
ncConfig1.dataIPAddress = "127.0.0.1";
+ ncConfig1.datasetIPAddress = "127.0.0.1";
ncConfig1.nodeId = NC1_ID;
nc1 = new NodeControllerService(ncConfig1);
nc1.start();
@@ -89,6 +100,7 @@
ncConfig2.ccPort = 39001;
ncConfig2.clusterNetIPAddress = "127.0.0.1";
ncConfig2.dataIPAddress = "127.0.0.1";
+ ncConfig2.datasetIPAddress = "127.0.0.1";
ncConfig2.nodeId = NC2_ID;
nc2 = new NodeControllerService(ncConfig2);
nc2.start();
@@ -107,7 +119,7 @@
cc.stop();
}
- protected void runTest(JobSpecification spec) throws Exception {
+ protected JobId executeTest(JobSpecification spec) throws Exception {
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(spec.toJSON().toString(2));
}
@@ -115,25 +127,72 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(jobId.toString());
}
- hcc.waitForCompletion(jobId);
- dumpOutputFiles();
+ return jobId;
}
- private void dumpOutputFiles() {
- if (LOGGER.isLoggable(Level.INFO)) {
- for (File f : outputFiles) {
- if (f.exists() && f.isFile()) {
- try {
- LOGGER.info("Reading file: " + f.getAbsolutePath() + " in test: " + getClass().getName());
- String data = FileUtils.readFileToString(f);
- LOGGER.info(data);
- } catch (IOException e) {
- LOGGER.info("Error reading file: " + f.getAbsolutePath());
- LOGGER.info(e.getMessage());
- }
+ protected void runTest(JobSpecification spec) throws Exception {
+ JobId jobId = executeTest(spec);
+ hcc.waitForCompletion(jobId);
+ }
+
+ protected List<String> readResults(JobSpecification spec, JobId jobId, ResultSetId resultSetId) throws Exception {
+ int nReaders = 1;
+ ByteBuffer resultBuffer = ByteBuffer.allocate(spec.getFrameSize());
+ resultBuffer.clear();
+
+ IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor(spec.getFrameSize());
+
+ IHyracksDataset hyracksDataset = new HyracksDataset(hcc, spec.getFrameSize(), nReaders);
+ IHyracksDatasetReader reader = hyracksDataset.createReader(jobId, resultSetId);
+
+ List<String> resultRecords = new ArrayList<String>();
+ ByteBufferInputStream bbis = new ByteBufferInputStream();
+
+ int readSize = reader.read(resultBuffer);
+
+ while (readSize > 0) {
+
+ try {
+ frameTupleAccessor.reset(resultBuffer);
+ for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
+ int start = frameTupleAccessor.getTupleStartOffset(tIndex);
+ int length = frameTupleAccessor.getTupleEndOffset(tIndex) - start;
+ bbis.setByteBuffer(resultBuffer, start);
+ byte[] recordBytes = new byte[length];
+ bbis.read(recordBytes, 0, length);
+ resultRecords.add(new String(recordBytes, 0, length));
}
+ } finally {
+ bbis.close();
}
+
+ resultBuffer.clear();
+ readSize = reader.read(resultBuffer);
}
+ return resultRecords;
+ }
+
+ protected boolean runTestAndCompareResults(JobSpecification spec, String[] expectedFileNames) throws Exception {
+ JobId jobId = executeTest(spec);
+
+ List<String> results;
+ for (int i = 0; i < expectedFileNames.length; i++) {
+ results = readResults(spec, jobId, spec.getResultSetIds().get(i));
+ BufferedReader expectedFile = new BufferedReader(new FileReader(expectedFileNames[i]));
+
+ String expectedLine, actualLine;
+ int j = 0;
+ while ((expectedLine = expectedFile.readLine()) != null) {
+ actualLine = results.get(j).trim();
+ Assert.assertEquals(expectedLine, actualLine);
+ j++;
+ }
+ Assert.assertEquals(j, results.size());
+ expectedFile.close();
+ }
+
+ hcc.waitForCompletion(jobId);
+ return true;
}
protected File createTempFile() throws IOException {
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
index e0b8c73..24d0ef4 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AbstractMultiNCIntegrationTest.java
@@ -16,6 +16,7 @@
import java.io.File;
import java.io.IOException;
+import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.EnumSet;
import java.util.List;
@@ -23,6 +24,7 @@
import java.util.logging.Logger;
import org.apache.commons.io.FileUtils;
+import org.json.JSONArray;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Rule;
@@ -30,13 +32,20 @@
import edu.uci.ics.hyracks.api.client.HyracksConnection;
import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
+import edu.uci.ics.hyracks.api.dataset.IHyracksDatasetReader;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
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.client.dataset.HyracksDataset;
import edu.uci.ics.hyracks.control.cc.ClusterControllerService;
import edu.uci.ics.hyracks.control.common.controllers.CCConfig;
import edu.uci.ics.hyracks.control.common.controllers.NCConfig;
import edu.uci.ics.hyracks.control.nc.NodeControllerService;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ResultFrameTupleAccessor;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
public abstract class AbstractMultiNCIntegrationTest {
@@ -84,6 +93,7 @@
ncConfig.ccPort = 39001;
ncConfig.clusterNetIPAddress = "127.0.0.1";
ncConfig.dataIPAddress = "127.0.0.1";
+ ncConfig.datasetIPAddress = "127.0.0.1";
ncConfig.nodeId = ASTERIX_IDS[i];
asterixNCs[i] = new NodeControllerService(ncConfig);
asterixNCs[i].start();
@@ -112,6 +122,46 @@
if (LOGGER.isLoggable(Level.INFO)) {
LOGGER.info(jobId.toString());
}
+
+ int nReaders = 1;
+
+ ByteBuffer resultBuffer = ByteBuffer.allocate(spec.getFrameSize());
+ resultBuffer.clear();
+
+ IFrameTupleAccessor frameTupleAccessor = new ResultFrameTupleAccessor(spec.getFrameSize());
+
+ IHyracksDataset hyracksDataset = new HyracksDataset(hcc, spec.getFrameSize(), nReaders);
+ IHyracksDatasetReader reader = hyracksDataset.createReader(jobId, spec.getResultSetIds().get(0));
+
+ JSONArray resultRecords = new JSONArray();
+ ByteBufferInputStream bbis = new ByteBufferInputStream();
+
+ int readSize = reader.read(resultBuffer);
+
+ while (readSize > 0) {
+
+ try {
+ frameTupleAccessor.reset(resultBuffer);
+ for (int tIndex = 0; tIndex < frameTupleAccessor.getTupleCount(); tIndex++) {
+ int start = frameTupleAccessor.getTupleStartOffset(tIndex);
+ int length = frameTupleAccessor.getTupleEndOffset(tIndex) - start;
+ bbis.setByteBuffer(resultBuffer, start);
+ byte[] recordBytes = new byte[length];
+ bbis.read(recordBytes, 0, length);
+ resultRecords.put(new String(recordBytes, 0, length));
+ }
+ } finally {
+ try {
+ bbis.close();
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+
+ resultBuffer.clear();
+ readSize = reader.read(resultBuffer);
+ }
+
hcc.waitForCompletion(jobId);
dumpOutputFiles();
}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
index 93e1e9b..06751e3 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/AggregationTest.java
@@ -25,8 +25,8 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
@@ -49,7 +49,6 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.group.HashSpillableTableFactory;
import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.AvgFieldGroupAggregatorFactory;
@@ -62,60 +61,42 @@
import edu.uci.ics.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.group.hash.HashGroupOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
/**
*
*/
public class AggregationTest extends AbstractIntegrationTest {
- final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(
- new FileSplit[] { new FileSplit(NC2_ID, new FileReference(new File(
- "data/tpch0.001/lineitem.tbl"))) });
+ final IFileSplitProvider splitProvider = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC2_ID,
+ new FileReference(new File("data/tpch0.001/lineitem.tbl"))) });
- final RecordDescriptor desc = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ final RecordDescriptor desc = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+ FloatSerializerDeserializer.INSTANCE, FloatSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
- final ITupleParserFactory tupleParserFactory = new DelimitedDataTupleParserFactory(
- new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE,
- IntegerParserFactory.INSTANCE,
- IntegerParserFactory.INSTANCE,
- IntegerParserFactory.INSTANCE,
- IntegerParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
- FloatParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE,
- UTF8StringParserFactory.INSTANCE, }, '|');
+ final ITupleParserFactory tupleParserFactory = new DelimitedDataTupleParserFactory(new IValueParserFactory[] {
+ UTF8StringParserFactory.INSTANCE, IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE,
+ IntegerParserFactory.INSTANCE, IntegerParserFactory.INSTANCE, FloatParserFactory.INSTANCE,
+ FloatParserFactory.INSTANCE, FloatParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE,
+ UTF8StringParserFactory.INSTANCE, }, '|');
- private AbstractSingleActivityOperatorDescriptor getPrinter(
- IOperatorDescriptorRegistry spec, String prefix) throws IOException {
+ private AbstractSingleActivityOperatorDescriptor getPrinter(JobSpecification spec, String prefix)
+ throws IOException {
- AbstractSingleActivityOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(
- spec, new ConstantFileSplitProvider(new FileSplit[] {
- new FileSplit(NC1_ID, createTempFile()
- .getAbsolutePath()),
- new FileSplit(NC2_ID, createTempFile()
- .getAbsolutePath()) }), "\t");
+ ResultSetId rsId = new ResultSetId(1);
+ AbstractSingleActivityOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ spec.addResultSetId(rsId);
return printer;
}
@@ -124,54 +105,38 @@
public void singleKeySumInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- 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,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeySumInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -184,49 +149,34 @@
public void singleKeySumPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- 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 };
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec,
- keyFields,
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new IntSumFieldAggregatorFactory(3, true),
- new FloatSumFieldAggregatorFactory(5, true)}),
- outputRec);
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true),
+ new FloatSumFieldAggregatorFactory(5, true) }), outputRec);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeySumInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -239,64 +189,43 @@
public void singleKeySumExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- 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 frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
new IntSumFieldAggregatorFactory(3, false),
- new FloatSumFieldAggregatorFactory(5, false)}),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
+ new FloatSumFieldAggregatorFactory(5, false) }), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
new IntSumFieldAggregatorFactory(2, false),
- new FloatSumFieldAggregatorFactory(3, false)}),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- tableSize), true);
+ new FloatSumFieldAggregatorFactory(3, false) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }), tableSize), true);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeySumExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeySumExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -309,54 +238,38 @@
public void singleKeyAvgInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- 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 CountFieldAggregatorFactory(true),
- new AvgFieldGroupAggregatorFactory(1, true) }),
- outputRec, tableSize);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+ new AvgFieldGroupAggregatorFactory(1, true) }), outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -369,49 +282,34 @@
public void singleKeyAvgPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- 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 };
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec,
- keyFields,
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new CountFieldAggregatorFactory(true),
- new AvgFieldGroupAggregatorFactory(1, true) }),
- outputRec);
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+ new AvgFieldGroupAggregatorFactory(1, true) }), outputRec);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -424,64 +322,43 @@
public void singleKeyAvgExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- 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 frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new CountFieldAggregatorFactory(false),
- new AvgFieldGroupAggregatorFactory(1, false) }),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, false), new CountFieldAggregatorFactory(false),
+ new AvgFieldGroupAggregatorFactory(1, false) }), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
new IntSumFieldAggregatorFactory(2, false),
- new AvgFieldMergeAggregatorFactory(3, false) }),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- tableSize), true);
+ new AvgFieldMergeAggregatorFactory(3, false) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }), tableSize), true);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -494,52 +371,38 @@
public void singleKeyMinMaxStringInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.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 MinMaxStringFieldAggregatorFactory(15,
- true, false) }), outputRec, tableSize);
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true),
+ new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -552,47 +415,34 @@
public void singleKeyMinMaxStringPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec,
- keyFields,
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new MinMaxStringFieldAggregatorFactory(15,
- true, false) }), outputRec);
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true),
+ new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -605,63 +455,42 @@
public void singleKeyMinMaxStringExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 0 };
int frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new MinMaxStringFieldAggregatorFactory(15,
- true, true) }),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new MinMaxStringFieldAggregatorFactory(2, true,
- true) }),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- tableSize), true);
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+ new MinMaxStringFieldAggregatorFactory(15, true, true) }),
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, false),
+ new MinMaxStringFieldAggregatorFactory(2, true, true) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
+ new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
+ .of(UTF8StringPointable.FACTORY) }), tableSize), true);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec,
- new FieldHashPartitionComputerFactory(
- keyFields,
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] { PointableBinaryHashFunctionFactory
.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "singleKeyAvgExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "singleKeyAvgExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -674,58 +503,39 @@
public void multiKeySumInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
int tableSize = 8;
- HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
- spec, keyFields, new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new IntSumFieldAggregatorFactory(3, true) }),
+ HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true) }),
outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeySumInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -738,51 +548,35 @@
public void multiKeySumPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec, keyFields,
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new IntSumFieldAggregatorFactory(3, true) }),
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new IntSumFieldAggregatorFactory(3, true) }),
outputRec);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeySumInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -795,69 +589,43 @@
public void multiKeySumExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
int frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new IntSumFieldAggregatorFactory(3, false) }),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(2, false),
- new IntSumFieldAggregatorFactory(3, false) }),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- tableSize), true);
-
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
-
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+ new IntSumFieldAggregatorFactory(3, false) }), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(2, false),
+ new IntSumFieldAggregatorFactory(3, false) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeySumExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeySumExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -870,60 +638,40 @@
public void multiKeyAvgInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
int tableSize = 8;
- HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
- spec, keyFields, new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new CountFieldAggregatorFactory(true),
- new AvgFieldGroupAggregatorFactory(1, true) }),
- outputRec, tableSize);
+ HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+ new AvgFieldGroupAggregatorFactory(1, true) }), outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyAvgInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -936,53 +684,36 @@
public void multiKeyAvgPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec, keyFields,
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new CountFieldAggregatorFactory(true),
- new AvgFieldGroupAggregatorFactory(1, true) }),
- outputRec);
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true), new CountFieldAggregatorFactory(true),
+ new AvgFieldGroupAggregatorFactory(1, true) }), outputRec);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyAvgInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -995,72 +726,46 @@
public void multiKeyAvgExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- FloatSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE,
+ FloatSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
int frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new CountFieldAggregatorFactory(false),
- new AvgFieldGroupAggregatorFactory(1, false) }),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(2, false),
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, false), new CountFieldAggregatorFactory(false),
+ new AvgFieldGroupAggregatorFactory(1, false) }), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(2, false),
new IntSumFieldAggregatorFactory(3, false),
- new AvgFieldMergeAggregatorFactory(4, false) }),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- tableSize), true);
-
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
-
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
+ new AvgFieldMergeAggregatorFactory(4, false) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyAvgExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyAvgExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -1073,58 +778,39 @@
public void multiKeyMinMaxStringInmemGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
int tableSize = 8;
- HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(
- spec, keyFields, new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new MinMaxStringFieldAggregatorFactory(15,
- true, false) }), outputRec, tableSize);
+ HashGroupOperatorDescriptor grouper = new HashGroupOperatorDescriptor(spec, keyFields,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }),
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true),
+ new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec, tableSize);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyMinMaxStringInmemGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringInmemGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -1137,51 +823,35 @@
public void multiKeyMinMaxStringPreClusterGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
- PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(
- spec, keyFields,
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, true),
- new MinMaxStringFieldAggregatorFactory(15,
- true, false) }), outputRec);
+ PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, keyFields,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(1, true),
+ new MinMaxStringFieldAggregatorFactory(15, true, false) }), outputRec);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyMinMaxStringPreClusterGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringPreClusterGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
@@ -1194,71 +864,44 @@
public void multiKeyMinMaxStringExtGroupTest() throws Exception {
JobSpecification spec = new JobSpecification();
- FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(
- spec, splitProvider, tupleParserFactory, desc);
+ FileScanOperatorDescriptor csvScanner = new FileScanOperatorDescriptor(spec, splitProvider, tupleParserFactory,
+ desc);
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec,
- csvScanner, NC2_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID);
- RecordDescriptor outputRec = new RecordDescriptor(
- new ISerializerDeserializer[] {
- UTF8StringSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE,
- IntegerSerializerDeserializer.INSTANCE,
- UTF8StringSerializerDeserializer.INSTANCE });
+ RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] {
+ UTF8StringSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE,
+ IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE });
int[] keyFields = new int[] { 8, 0 };
int frameLimits = 4;
int tableSize = 8;
- ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(
- spec,
- keyFields,
- frameLimits,
- new IBinaryComparatorFactory[] {
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryComparatorFactory
- .of(UTF8StringPointable.FACTORY) },
- new UTF8StringNormalizedKeyComputerFactory(),
- new MultiFieldsAggregatorFactory(
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(1, false),
- new MinMaxStringFieldAggregatorFactory(15,
- true, true) }),
- new MultiFieldsAggregatorFactory(new int[] { 0, 1 },
- new IFieldAggregateDescriptorFactory[] {
- new IntSumFieldAggregatorFactory(2, false),
- new MinMaxStringFieldAggregatorFactory(3, true,
- true) }),
- outputRec,
- new HashSpillableTableFactory(
- new FieldHashPartitionComputerFactory(
- keyFields,
- new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }),
- tableSize), true);
-
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper,
- NC2_ID, NC1_ID);
-
- IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(
- spec, new FieldHashPartitionComputerFactory(keyFields,
+ ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, keyFields, frameLimits,
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
+ new UTF8StringNormalizedKeyComputerFactory(), new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, false),
+ new MinMaxStringFieldAggregatorFactory(15, true, true) }),
+ new MultiFieldsAggregatorFactory(new int[] { 0, 1 }, new IFieldAggregateDescriptorFactory[] {
+ new IntSumFieldAggregatorFactory(2, false),
+ new MinMaxStringFieldAggregatorFactory(3, true, true) }), outputRec,
+ new HashSpillableTableFactory(new FieldHashPartitionComputerFactory(keyFields,
new IBinaryHashFunctionFactory[] {
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY),
- PointableBinaryHashFunctionFactory
- .of(UTF8StringPointable.FACTORY) }));
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), tableSize), true);
+
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC2_ID, NC1_ID);
+
+ IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
+ new FieldHashPartitionComputerFactory(keyFields, new IBinaryHashFunctionFactory[] {
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY),
+ PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }));
spec.connect(conn1, csvScanner, 0, grouper, 0);
- AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec,
- "multiKeyMinMaxStringExtGroupTest");
+ AbstractSingleActivityOperatorDescriptor printer = getPrinter(spec, "multiKeyMinMaxStringExtGroupTest");
- PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer,
- NC2_ID, NC1_ID);
+ PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn2 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn2, grouper, 0, printer, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
index aea6126..5008991 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/CountOfCountsTest.java
@@ -25,6 +25,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -44,13 +45,14 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.InMemorySortOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class CountOfCountsTest extends AbstractIntegrationTest {
@Test
@@ -76,12 +78,10 @@
RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
- PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
- spec,
- new int[] { 0 },
+ PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 0 },
new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
- desc2);
+ new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc2);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID);
InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
@@ -91,13 +91,15 @@
RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(
new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ spec.addResultSetId(rsId);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
@@ -148,12 +150,10 @@
RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
- PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
- spec,
- new int[] { 0 },
+ PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 0 },
new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
- desc2);
+ new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc2);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
@@ -163,13 +163,16 @@
RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(
new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ spec.addResultSetId(rsId);
+
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
@@ -220,12 +223,10 @@
RecordDescriptor desc2 = new RecordDescriptor(new ISerializerDeserializer[] {
UTF8StringSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
- PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(
- spec,
- new int[] { 0 },
+ PreclusteredGroupOperatorDescriptor group = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 0 },
new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) },
- new MultiFieldsAggregatorFactory(new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }),
- desc2);
+ new MultiFieldsAggregatorFactory(
+ new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc2);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group, NC1_ID, NC2_ID, NC1_ID, NC2_ID);
InMemorySortOperatorDescriptor sorter2 = new InMemorySortOperatorDescriptor(spec, new int[] { 1 },
@@ -235,13 +236,16 @@
RecordDescriptor desc3 = new RecordDescriptor(new ISerializerDeserializer[] {
IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE });
PreclusteredGroupOperatorDescriptor group2 = new PreclusteredGroupOperatorDescriptor(spec, new int[] { 1 },
- new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, new MultiFieldsAggregatorFactory(
+ new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) },
+ new MultiFieldsAggregatorFactory(
new IFieldAggregateDescriptorFactory[] { new CountFieldAggregatorFactory(true) }), desc3);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, group2, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ spec.addResultSetId(rsId);
+
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java
index 0d5a627..93ed2c7 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/LocalityAwareConnectorTest.java
@@ -26,8 +26,8 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
@@ -52,12 +52,13 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.FloatSumFieldAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory;
import edu.uci.ics.hyracks.dataflow.std.group.hash.HashGroupOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class LocalityAwareConnectorTest extends AbstractMultiNCIntegrationTest {
@@ -200,13 +201,13 @@
runTest(spec);
}
- private AbstractSingleActivityOperatorDescriptor getPrinter(IOperatorDescriptorRegistry spec, String prefix)
+ private AbstractSingleActivityOperatorDescriptor getPrinter(JobSpecification 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");
+ ResultSetId rsId = new ResultSetId(1);
+ AbstractSingleActivityOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ spec.addResultSetId(rsId);
return printer;
}
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
index 1ee4400..ec9be32 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/OptimizedSortMergeTest.java
@@ -24,6 +24,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -40,9 +41,10 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.LimitOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.OptimizedExternalSortOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class OptimizedSortMergeTest extends AbstractIntegrationTest {
@@ -75,9 +77,11 @@
PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -128,9 +132,11 @@
LimitOperatorDescriptor filter = new LimitOperatorDescriptor(spec, ordersDesc, outputLimit);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, filter, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
index 9355110..961f780 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/ScanPrintTest.java
@@ -24,6 +24,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
@@ -42,7 +43,8 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class ScanPrintTest extends AbstractIntegrationTest {
@Test
@@ -63,10 +65,11 @@
desc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, csvScanner, NC2_ID, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] {
- new FileSplit(NC2_ID, createTempFile().getAbsolutePath()),
- new FileSplit(NC1_ID, createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ spec.addResultSetId(rsId);
+
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
IConnectorDescriptor conn = new OneToOneConnectorDescriptor(spec);
@@ -98,9 +101,11 @@
UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ spec.addResultSetId(rsId);
+
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
@@ -135,9 +140,11 @@
UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
+ spec.addResultSetId(rsId);
+
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor conn1 = new MToNPartitioningConnectorDescriptor(spec,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
index 2c3fddf..0da93f2 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SortMergeTest.java
@@ -24,6 +24,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -40,9 +41,10 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.InMemorySortOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class SortMergeTest extends AbstractIntegrationTest {
@Test
@@ -73,9 +75,11 @@
ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
@@ -118,9 +122,11 @@
PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorter, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorter, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SplitOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SplitOperatorTest.java
index 2b32142..6040748 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SplitOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/SplitOperatorTest.java
@@ -26,6 +26,7 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
@@ -35,8 +36,9 @@
import edu.uci.ics.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-import edu.uci.ics.hyracks.dataflow.std.file.LineFileWriteOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.SplitOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class SplitOperatorTest extends AbstractIntegrationTest {
@@ -50,6 +52,8 @@
Assert.assertEquals(lineA, lineB);
}
Assert.assertNull(fileB.readLine());
+ fileA.close();
+ fileB.close();
}
@Test
@@ -83,8 +87,11 @@
IOperatorDescriptor outputOp[] = new IOperatorDescriptor[outputFile.length];
for (int i = 0; i < outputArity; i++) {
- outputOp[i] = new LineFileWriteOperatorDescriptor(spec, new FileSplit[] { new FileSplit(NC1_ID,
- outputFile[i].getAbsolutePath()) });
+ ResultSetId rsId = new ResultSetId(i);
+ spec.addResultSetId(rsId);
+
+ outputOp[i] = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, outputOp[i], locations);
}
@@ -96,10 +103,10 @@
for (int i = 0; i < outputArity; i++) {
spec.addRoot(outputOp[i]);
}
- runTest(spec);
-
+ String[] expectedResultsFileNames = new String[outputArity];
for (int i = 0; i < outputArity; i++) {
- compareFiles(inputFileName, outputFile[i].getAbsolutePath());
+ expectedResultsFileNames[i] = inputFileName;
}
+ runTestAndCompareResults(spec, expectedResultsFileNames);
}
}
\ No newline at end of file
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
index 622942b..b5eb850 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
@@ -14,9 +14,7 @@
*/
package edu.uci.ics.hyracks.tests.integration;
-import java.io.DataOutput;
import java.io.File;
-import java.io.IOException;
import org.junit.Test;
@@ -25,11 +23,10 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.INullWriter;
import edu.uci.ics.hyracks.api.dataflow.value.INullWriterFactory;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -47,12 +44,13 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.join.GraceHashJoinOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.join.HybridHashJoinOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.join.InMemoryHashJoinOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.MaterializingOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.tests.util.NoopNullWriterFactory;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class TPCHCustomerOrderHashJoinTest extends AbstractIntegrationTest {
@@ -127,9 +125,11 @@
custOrderJoinDesc, 128);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -208,9 +208,11 @@
custOrderJoinDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -289,9 +291,11 @@
custOrderJoinDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -371,9 +375,11 @@
custOrderJoinDesc, true, nullWriterFactories, 128);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -457,9 +463,11 @@
custOrderJoinDesc, true, nullWriterFactories);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -543,9 +551,11 @@
custOrderJoinDesc, true, nullWriterFactories);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -622,9 +632,11 @@
custOrderJoinDesc, 128);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
@@ -711,9 +723,11 @@
custOrderJoinDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
@@ -800,9 +814,11 @@
custOrderJoinDesc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
@@ -885,9 +901,11 @@
custOrderJoinDesc, 128);
PartitionConstraintHelper.addPartitionCountConstraint(spec, join, 2);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new MToNPartitioningConnectorDescriptor(spec,
@@ -976,9 +994,11 @@
custOrderJoinDesc, 128);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordPartConn = new MToNPartitioningConnectorDescriptor(spec,
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
index 9233e39..99f2d18 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
@@ -15,7 +15,9 @@
package edu.uci.ics.hyracks.tests.integration;
import java.io.File;
+
import org.junit.Test;
+
import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -28,6 +30,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparator;
import edu.uci.ics.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -42,9 +45,10 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.join.NestedLoopJoinOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.tests.util.NoopNullWriterFactory;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class TPCHCustomerOrderNestedLoopJoinTest extends AbstractIntegrationTest {
private static class JoinComparatorFactory implements ITuplePairComparatorFactory {
@@ -169,9 +173,11 @@
null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -244,9 +250,11 @@
null);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -319,9 +327,11 @@
null);
PartitionConstraintHelper.addPartitionCountConstraint(spec, join, 2);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
@@ -399,9 +409,11 @@
nullWriterFactories);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, join, NC1_ID, NC2_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/UnionTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/UnionTest.java
index 37b55b8..5b323e6 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/UnionTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/integration/UnionTest.java
@@ -22,6 +22,7 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
@@ -33,8 +34,9 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.union.UnionAllOperatorDescriptor;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class UnionTest extends AbstractIntegrationTest {
@Test
@@ -65,10 +67,11 @@
UnionAllOperatorDescriptor unionAll = new UnionAllOperatorDescriptor(spec, 2, desc);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, unionAll, NC2_ID, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] {
- new FileSplit(NC2_ID, createTempFile().getAbsolutePath()),
- new FileSplit(NC1_ID, createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC2_ID, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), csvScanner01, 0, unionAll, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
index 836e72e..600b54b 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/BinaryTokenizerOperatorTest.java
@@ -8,6 +8,7 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
@@ -21,13 +22,14 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class BinaryTokenizerOperatorTest extends AbstractIntegrationTest {
@@ -58,9 +60,11 @@
tokenizerRecDesc, tokenizerFactory, tokenFields, keyFields);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, binaryTokenizer, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), dblpTitleScanner, 0, binaryTokenizer, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
index 2206a26..b5a4df8 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/InvertedIndexOperatorsTest.java
@@ -8,6 +8,7 @@
import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
@@ -21,13 +22,14 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class InvertedIndexOperatorsTest extends AbstractIntegrationTest {
@@ -58,9 +60,11 @@
tokenizerRecDesc, tokenizerFactory, tokenFields, projFields);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, binaryTokenizer, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), dblpTitleScanner, 0, binaryTokenizer, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
index d8fd48e..d1071a3 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/invertedindex/WordInvertedIndexTest.java
@@ -30,6 +30,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
@@ -48,8 +49,8 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -74,6 +75,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class WordInvertedIndexTest extends AbstractIntegrationTest {
static {
@@ -87,9 +89,12 @@
private final static SimpleDateFormat simpleDateFormat = new SimpleDateFormat("ddMMyy-hhmmssSS");
private final static String sep = System.getProperty("file.separator");
private final static String dateString = simpleDateFormat.format(new Date());
- private final static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + "primaryBtree" + dateString;
- private final static String btreeFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexBtree" + dateString;
- private final static String invListsFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexLists" + dateString;
+ private final static String primaryFileName = System.getProperty("java.io.tmpdir") + sep + "primaryBtree"
+ + dateString;
+ private final static String btreeFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexBtree"
+ + dateString;
+ private final static String invListsFileName = System.getProperty("java.io.tmpdir") + sep + "invIndexLists"
+ + dateString;
private IFileSplitProvider primaryFileSplitProvider = new ConstantFileSplitProvider(
new FileSplit[] { new FileSplit(NC1_ID, new FileReference(new File(primaryFileName))) });
@@ -155,7 +160,7 @@
spec.addRoot(primaryCreateOp);
runTest(spec);
}
-
+
@Test
public void testConjunctiveSearcher() throws Exception {
IInvertedIndexSearchModifierFactory conjunctiveSearchModifierFactory = new ConjunctiveSearchModifierFactory();
@@ -180,8 +185,9 @@
private IOperatorDescriptor createPrimaryBulkLoadOp(JobSpecification spec) {
int[] fieldPermutation = { 0, 1 };
TreeIndexBulkLoadOperatorDescriptor primaryBtreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
- storageManager, indexRegistryProvider, primaryFileSplitProvider, primaryTypeTraits, primaryComparatorFactories, fieldPermutation, 0.7f,
- btreeDataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryFileSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, fieldPermutation, 0.7f, btreeDataflowHelperFactory,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeBulkLoad, NC1_ID);
return primaryBtreeBulkLoad;
}
@@ -206,8 +212,9 @@
int[] lowKeyFields = null; // - infinity
int[] highKeyFields = null; // + infinity
BTreeSearchOperatorDescriptor primaryBtreeSearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
- storageManager, indexRegistryProvider, primaryFileSplitProvider, primaryTypeTraits, primaryComparatorFactories, lowKeyFields,
- highKeyFields, true, true, btreeDataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, indexRegistryProvider, primaryFileSplitProvider, primaryTypeTraits,
+ primaryComparatorFactories, lowKeyFields, highKeyFields, true, true, btreeDataflowHelperFactory, false,
+ NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryBtreeSearchOp, NC1_ID);
return primaryBtreeSearchOp;
}
@@ -227,9 +234,12 @@
JobSpecification spec = new JobSpecification();
IOperatorDescriptor keyProviderOp = createScanKeyProviderOp(spec);
IOperatorDescriptor primaryScanOp = createPrimaryScanOp(spec);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, printer, 0);
spec.addRoot(printer);
@@ -265,15 +275,14 @@
public void createInvertedIndex() throws Exception {
JobSpecification spec = new JobSpecification();
InvertedIndexCreateOperatorDescriptor invIndexCreateOp = new InvertedIndexCreateOperatorDescriptor(spec,
- storageManager, btreeFileSplitProvider, invListsFileSplitProvider,
- indexRegistryProvider, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
- invListsComparatorFactories, tokenizerFactory, btreeDataflowHelperFactory,
- NoOpOperationCallbackProvider.INSTANCE);
+ storageManager, btreeFileSplitProvider, invListsFileSplitProvider, indexRegistryProvider,
+ tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits, invListsComparatorFactories,
+ tokenizerFactory, btreeDataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, invIndexCreateOp, NC1_ID);
spec.addRoot(invIndexCreateOp);
runTest(spec);
}
-
+
public void loadInvertedIndex() throws Exception {
JobSpecification spec = new JobSpecification();
IOperatorDescriptor keyProviderOp = createScanKeyProviderOp(spec);
@@ -325,20 +334,23 @@
JobSpecification spec = new JobSpecification();
IOperatorDescriptor queryProviderOp = createQueryProviderOp(spec, queryString);
IOperatorDescriptor invIndexSearchOp = createInvertedIndexSearchOp(spec, searchModifierFactory);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
spec.connect(new OneToOneConnectorDescriptor(spec), queryProviderOp, 0, invIndexSearchOp, 0);
spec.connect(new OneToOneConnectorDescriptor(spec), invIndexSearchOp, 0, printer, 0);
spec.addRoot(printer);
runTest(spec);
}
-
+
@AfterClass
public static void cleanup() throws Exception {
- File primary = new File(primaryFileName);
- File btree = new File(btreeFileName);
- File invLists = new File(invListsFileName);
+ File primary = new File(primaryFileName);
+ File btree = new File(btreeFileName);
+ File invLists = new File(invListsFileName);
primary.deleteOnExit();
btree.deleteOnExit();
invLists.deleteOnExit();
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
index 6625148..92b6e14 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexSearchOperatorTest.java
@@ -30,6 +30,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -47,8 +48,8 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
@@ -64,6 +65,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class RTreePrimaryIndexSearchOperatorTest extends AbstractIntegrationTest {
static {
@@ -190,9 +192,11 @@
dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryRTreeSearchOp, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryRTreeSearchOp, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
index ef2950e..2a00394 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreePrimaryIndexStatsOperatorTest.java
@@ -29,6 +29,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -45,6 +46,7 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
@@ -64,6 +66,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class RTreePrimaryIndexStatsOperatorTest extends AbstractIntegrationTest {
static {
@@ -175,9 +178,11 @@
primaryTypeTraits, primaryComparatorFactories, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, primaryStatsOp, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), primaryStatsOp, 0, printer, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
index 030afcf..1d86037 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/rtree/RTreeSecondaryIndexSearchOperatorTest.java
@@ -30,6 +30,7 @@
import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.dataset.ResultSetId;
import edu.uci.ics.hyracks.api.io.FileReference;
import edu.uci.ics.hyracks.api.job.JobSpecification;
import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -47,8 +48,8 @@
import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.dataflow.std.file.PlainFileWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor;
import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -67,6 +68,7 @@
import edu.uci.ics.hyracks.test.support.TestStorageManagerComponentHolder;
import edu.uci.ics.hyracks.test.support.TestStorageManagerInterface;
import edu.uci.ics.hyracks.tests.integration.AbstractIntegrationTest;
+import edu.uci.ics.hyracks.tests.util.ResultSerializerFactoryProvider;
public class RTreeSecondaryIndexSearchOperatorTest extends AbstractIntegrationTest {
static {
@@ -297,9 +299,11 @@
secondaryTypeTraits, secondaryComparatorFactories, keyFields, dataflowHelperFactory, false, NoOpOperationCallbackProvider.INSTANCE);
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondaryRTreeSearchOp, NC1_ID);
- IFileSplitProvider outSplits = new ConstantFileSplitProvider(new FileSplit[] { new FileSplit(NC1_ID,
- createTempFile().getAbsolutePath()) });
- IOperatorDescriptor printer = new PlainFileWriterOperatorDescriptor(spec, outSplits, ",");
+ ResultSetId rsId = new ResultSetId(1);
+ spec.addResultSetId(rsId);
+
+ IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, true,
+ ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider());
PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID);
spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondaryRTreeSearchOp, 0);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/ResultSerializerFactoryProvider.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/ResultSerializerFactoryProvider.java
new file mode 100644
index 0000000..19c4475
--- /dev/null
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/edu/uci/ics/hyracks/tests/util/ResultSerializerFactoryProvider.java
@@ -0,0 +1,76 @@
+/*
+ * 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.tests.util;
+
+import java.io.DataInputStream;
+import java.io.PrintStream;
+import java.io.Serializable;
+
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IResultSerializerFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+
+public class ResultSerializerFactoryProvider implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ public static final ResultSerializerFactoryProvider INSTANCE = new ResultSerializerFactoryProvider();
+
+ private ResultSerializerFactoryProvider() {
+ }
+
+ public IResultSerializerFactory getResultSerializerFactoryProvider() {
+ return new IResultSerializerFactory() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IResultSerializer createResultSerializer(final RecordDescriptor recordDesc,
+ final PrintStream printStream) {
+ return new IResultSerializer() {
+ private static final long serialVersionUID = 1L;
+
+ ByteBufferInputStream bbis = new ByteBufferInputStream();
+ DataInputStream di = new DataInputStream(bbis);
+
+ @Override
+ public void init() throws HyracksDataException {
+
+ }
+
+ @Override
+ public boolean appendTuple(IFrameTupleAccessor tAccess, int tIdx) throws HyracksDataException {
+ int start = tAccess.getTupleStartOffset(tIdx) + tAccess.getFieldSlotsLength();
+
+ bbis.setByteBuffer(tAccess.getBuffer(), start);
+
+ Object[] record = new Object[recordDesc.getFieldCount()];
+ for (int i = 0; i < record.length; ++i) {
+ Object instance = recordDesc.getFields()[i].deserialize(di);
+ if (i == 0) {
+ printStream.print(String.valueOf(instance));
+ } else {
+ printStream.print(", " + String.valueOf(instance));
+ }
+ }
+ printStream.println();
+ return true;
+ }
+ };
+ }
+ };
+ }
+}
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/scheduler/SchedulerTest.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/scheduler/SchedulerTest.java
index cc1a299..e42e8a6 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/scheduler/SchedulerTest.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/scheduler/SchedulerTest.java
@@ -41,17 +41,23 @@
public void testSchedulerSimple() throws Exception {
Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.1").getAddress(), 5099)));
+ .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.2").getAddress(), 5099)));
+ .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.3").getAddress(), 5099)));
+ .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.4").getAddress(), 5099)));
+ .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.5").getAddress(), 5099)));
+ .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.6").getAddress(), 5099)));
+ .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+ .getAddress(), 5098)));
InputSplit[] fileSplits = new InputSplit[6];
fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
@@ -79,17 +85,23 @@
public void testSchedulerLargerHDFS() throws Exception {
Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.1").getAddress(), 5099)));
+ .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.2").getAddress(), 5099)));
+ .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.3").getAddress(), 5099)));
+ .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.4").getAddress(), 5099)));
+ .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.5").getAddress(), 5099)));
+ .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.6").getAddress(), 5099)));
+ .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+ .getAddress(), 5098)));
InputSplit[] fileSplits = new InputSplit[12];
fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
@@ -124,17 +136,23 @@
public void testSchedulerSmallerHDFS() throws Exception {
Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.1").getAddress(), 5099)));
+ .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.2").getAddress(), 5099)));
+ .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.3").getAddress(), 5099)));
+ .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.4").getAddress(), 5099)));
+ .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.5").getAddress(), 5099)));
+ .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.6").getAddress(), 5099)));
+ .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+ .getAddress(), 5098)));
InputSplit[] fileSplits = new InputSplit[12];
fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
@@ -169,17 +187,23 @@
public void testSchedulerSmallerHDFSOdd() throws Exception {
Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.1").getAddress(), 5099)));
+ .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.2").getAddress(), 5099)));
+ .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.3").getAddress(), 5099)));
+ .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.4").getAddress(), 5099)));
+ .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.5").getAddress(), 5099)));
+ .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.6").getAddress(), 5099)));
+ .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+ .getAddress(), 5098)));
InputSplit[] fileSplits = new InputSplit[13];
fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java
index d44b75a..8c12518 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs/utils/HyracksUtils.java
@@ -64,6 +64,7 @@
ncConfig1.clusterNetIPAddress = "localhost";
ncConfig1.ccPort = TEST_HYRACKS_CC_PORT;
ncConfig1.dataIPAddress = "127.0.0.1";
+ ncConfig1.datasetIPAddress = "127.0.0.1";
ncConfig1.nodeId = NC1_ID;
nc1 = new NodeControllerService(ncConfig1);
nc1.start();
@@ -73,6 +74,7 @@
ncConfig2.clusterNetIPAddress = "localhost";
ncConfig2.ccPort = TEST_HYRACKS_CC_PORT;
ncConfig2.dataIPAddress = "127.0.0.1";
+ ncConfig2.datasetIPAddress = "127.0.0.1";
ncConfig2.nodeId = NC2_ID;
nc2 = new NodeControllerService(ncConfig2);
nc2.start();
diff --git a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/scheduler/SchedulerTest.java b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/scheduler/SchedulerTest.java
index 2fbeb7c..b5d9d1a 100644
--- a/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/scheduler/SchedulerTest.java
+++ b/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/edu/uci/ics/hyracks/hdfs2/scheduler/SchedulerTest.java
@@ -45,17 +45,23 @@
public void testSchedulerSimple() throws Exception {
Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.1").getAddress(), 5099)));
+ .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.2").getAddress(), 5099)));
+ .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.3").getAddress(), 5099)));
+ .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.4").getAddress(), 5099)));
+ .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.5").getAddress(), 5099)));
+ .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.6").getAddress(), 5099)));
+ .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+ .getAddress(), 5098)));
List<InputSplit> fileSplits = new ArrayList<InputSplit>();
fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
@@ -83,17 +89,23 @@
public void testSchedulerLargerHDFS() throws Exception {
Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.1").getAddress(), 5099)));
+ .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.2").getAddress(), 5099)));
+ .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.3").getAddress(), 5099)));
+ .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.4").getAddress(), 5099)));
+ .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.5").getAddress(), 5099)));
+ .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.6").getAddress(), 5099)));
+ .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+ .getAddress(), 5098)));
List<InputSplit> fileSplits = new ArrayList<InputSplit>();
fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
@@ -128,17 +140,23 @@
public void testSchedulerSmallerHDFS() throws Exception {
Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.1").getAddress(), 5099)));
+ .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.2").getAddress(), 5099)));
+ .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.3").getAddress(), 5099)));
+ .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.4").getAddress(), 5099)));
+ .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.5").getAddress(), 5099)));
+ .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.6").getAddress(), 5099)));
+ .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+ .getAddress(), 5098)));
List<InputSplit> fileSplits = new ArrayList<InputSplit>();
fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
@@ -173,17 +191,23 @@
public void testSchedulerSmallerHDFSOdd() throws Exception {
Map<String, NodeControllerInfo> ncNameToNcInfos = new HashMap<String, NodeControllerInfo>();
ncNameToNcInfos.put("nc1", new NodeControllerInfo("nc1", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.1").getAddress(), 5099)));
+ .getByName("10.0.0.1").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.1")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc2", new NodeControllerInfo("nc2", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.2").getAddress(), 5099)));
+ .getByName("10.0.0.2").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.2")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc3", new NodeControllerInfo("nc3", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.3").getAddress(), 5099)));
+ .getByName("10.0.0.3").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.3")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc4", new NodeControllerInfo("nc4", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.4").getAddress(), 5099)));
+ .getByName("10.0.0.4").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.4")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc5", new NodeControllerInfo("nc5", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.5").getAddress(), 5099)));
+ .getByName("10.0.0.5").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.5")
+ .getAddress(), 5098)));
ncNameToNcInfos.put("nc6", new NodeControllerInfo("nc6", NodeStatus.ALIVE, new NetworkAddress(InetAddress
- .getByName("10.0.0.6").getAddress(), 5099)));
+ .getByName("10.0.0.6").getAddress(), 5099), new NetworkAddress(InetAddress.getByName("10.0.0.6")
+ .getAddress(), 5098)));
List<InputSplit> fileSplits = new ArrayList<InputSplit>();
fileSplits.add(new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" }));
diff --git a/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksNCStartMojo.java b/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksNCStartMojo.java
index 47de024..fc06a68 100644
--- a/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksNCStartMojo.java
+++ b/hyracks/hyracks-maven-plugins/hyracks-virtualcluster-maven-plugin/src/main/java/edu/uci/ics/hyracks/maven/plugin/HyracksNCStartMojo.java
@@ -55,6 +55,7 @@
cmdLineBuffer.append(" -data-ip-address ").append(dataIpAddress);
cmdLineBuffer.append(" -node-id ").append(nodeId);
cmdLineBuffer.append(" -cluster-net-ip-address 127.0.0.1");
+ cmdLineBuffer.append(" -result-ip-address 127.0.0.1");
if (ccPort != 0) {
cmdLineBuffer.append(" -cc-port ").append(ccPort);
}
diff --git a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java
index c719bc4..e4df6b9 100644
--- a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java
+++ b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/muxdemux/MuxDemux.java
@@ -48,9 +48,9 @@
* Constructor.
*
* @param localAddress
- * - TCP/IP socket address to listen on
+ * - TCP/IP socket address to listen on. Null for non-listening unidirectional sockets
* @param listener
- * - Callback interface to report channel events
+ * - Callback interface to report channel events. Null for non-listening unidirectional sockets
* @param nThreads
* - Number of threads to use for data transfer
* @param maxConnectionAttempts
diff --git a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java
index d13a17e..a9061e1 100644
--- a/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java
+++ b/hyracks/hyracks-net/src/main/java/edu/uci/ics/hyracks/net/protocols/tcp/TCPEndpoint.java
@@ -45,15 +45,23 @@
}
public void start(InetSocketAddress localAddress) throws IOException {
- serverSocketChannel = ServerSocketChannel.open();
- ServerSocket serverSocket = serverSocketChannel.socket();
- serverSocket.bind(localAddress);
- this.localAddress = (InetSocketAddress) serverSocket.getLocalSocketAddress();
+ // Setup a server socket listening channel only if the TCPEndpoint is a listening endpoint.
+ if (localAddress != null) {
+ serverSocketChannel = ServerSocketChannel.open();
+ ServerSocket serverSocket = serverSocketChannel.socket();
+ serverSocket.bind(localAddress);
+ this.localAddress = (InetSocketAddress) serverSocket.getLocalSocketAddress();
+ }
+
ioThreads = new IOThread[nThreads];
for (int i = 0; i < ioThreads.length; ++i) {
ioThreads[i] = new IOThread();
}
- ioThreads[0].registerServerSocket(serverSocketChannel);
+
+ if (localAddress != null) {
+ ioThreads[0].registerServerSocket(serverSocketChannel);
+ }
+
for (int i = 0; i < ioThreads.length; ++i) {
ioThreads[i].start();
}
diff --git a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
index c122b25..0ca93b2 100644
--- a/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
+++ b/hyracks/hyracks-test-support/src/main/java/edu/uci/ics/hyracks/test/support/TestTaskContext.java
@@ -20,6 +20,7 @@
import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
import edu.uci.ics.hyracks.api.dataflow.TaskAttemptId;
import edu.uci.ics.hyracks.api.dataflow.state.IStateObject;
+import edu.uci.ics.hyracks.api.dataset.IDatasetPartitionManager;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.exceptions.HyracksException;
import edu.uci.ics.hyracks.api.io.FileReference;
@@ -101,6 +102,11 @@
}
@Override
+ public IDatasetPartitionManager getDatasetPartitionManager() {
+ return null;
+ }
+
+ @Override
public void sendApplicationMessageToCC(byte[] message, String nodeId) throws Exception {
// TODO Auto-generated method stub
diff --git a/hyracks/pom.xml b/hyracks/pom.xml
index 09925fb..b699542 100644
--- a/hyracks/pom.xml
+++ b/hyracks/pom.xml
@@ -83,6 +83,8 @@
<modules>
<module>hyracks-ipc</module>
<module>hyracks-api</module>
+ <module>hyracks-comm</module>
+ <module>hyracks-client</module>
<module>hyracks-dataflow-common</module>
<module>hyracks-dataflow-std</module>
<module>hyracks-dataflow-hadoop</module>
diff --git a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
index 2a2e2bf..c343763 100644
--- a/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
+++ b/pregelix/pregelix-core/src/main/java/edu/uci/ics/pregelix/core/util/PregelixHyracksIntegrationUtil.java
@@ -64,6 +64,7 @@
ncConfig1.clusterNetIPAddress = "localhost";
ncConfig1.ccPort = TEST_HYRACKS_CC_PORT;
ncConfig1.dataIPAddress = "127.0.0.1";
+ ncConfig1.datasetIPAddress = "127.0.0.1";
ncConfig1.nodeId = NC1_ID;
nc1 = new NodeControllerService(ncConfig1);
nc1.start();
@@ -73,6 +74,7 @@
ncConfig2.clusterNetIPAddress = "localhost";
ncConfig2.ccPort = TEST_HYRACKS_CC_PORT;
ncConfig2.dataIPAddress = "127.0.0.1";
+ ncConfig2.datasetIPAddress = "127.0.0.1";
ncConfig2.nodeId = NC2_ID;
nc2 = new NodeControllerService(ncConfig2);
nc2.start();