You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Preston Carman (Code Review)" <do...@asterixdb.incubator.apache.org> on 2015/10/16 01:19:55 UTC
Change in hyracks[master]: Adding merge join and updated the range connectors.
Preston Carman has uploaded a new change for review.
https://asterix-gerrit.ics.uci.edu/456
Change subject: Adding merge join and updated the range connectors.
......................................................................
Adding merge join and updated the range connectors.
Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
M algebricks/algebricks-examples/piglet-example/pom.xml
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
M hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java
M hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/ByteArrayPointableTest.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTuplePairComparator.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFamily.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
A hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopHashTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopNewPartitionerTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopPartitionerTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/util/InputSplitsProxy.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/GroupingHashTable.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameTupleBufferAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/util/PartitionUtil.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
M pom.xml
74 files changed, 2,320 insertions(+), 456 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/56/456/1
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
index c581e82..05685dd 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -60,6 +60,7 @@
SINK_WRITE,
SORT_GROUP_BY,
SORT_MERGE_EXCHANGE,
+ SORT_MERGE_INTERVAL_JOIN,
SPLIT,
STABLE_SORT,
STATS,
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
index eaeeac7..d254479 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
@@ -20,7 +20,7 @@
public class IndexedNLJoinExpressionAnnotation implements IExpressionAnnotation {
- public static final String INDEXED_NL_JOIN_ANNOTATION_KEY = "indexnl";
+ public static final String HINT_STRING = "indexnl";
public static final IndexedNLJoinExpressionAnnotation INSTANCE = new IndexedNLJoinExpressionAnnotation();
private Object object;
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index bfb9036..59758b5 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -66,8 +66,8 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.HashPartitionExchangePOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.HashPartitionMergeExchangePOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangePartitionMergePOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangePartitionPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangePartitionMergeExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangePartitionExchangePOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.SortMergeExchangePOperator;
import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
@@ -153,14 +153,14 @@
break;
}
case RANGE_PARTITION_EXCHANGE: {
- RangePartitionPOperator concreteOp = (RangePartitionPOperator) physOp;
+ RangePartitionExchangePOperator concreteOp = (RangePartitionExchangePOperator) physOp;
for (OrderColumn partCol : concreteOp.getPartitioningFields()) {
usedVariables.add(partCol.getColumn());
}
break;
}
case RANGE_PARTITION_MERGE_EXCHANGE: {
- RangePartitionMergePOperator concreteOp = (RangePartitionMergePOperator) physOp;
+ RangePartitionMergeExchangePOperator concreteOp = (RangePartitionMergeExchangePOperator) physOp;
for (OrderColumn partCol : concreteOp.getPartitioningFields()) {
usedVariables.add(partCol.getColumn());
}
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastPOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
similarity index 91%
rename from algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastPOperator.java
rename to algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
index c0e20d6..dd4b8c9 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
@@ -35,13 +35,13 @@
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
-import org.apache.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.connectors.MToNBroadcastConnectorDescriptor;
-public class BroadcastPOperator extends AbstractExchangePOperator {
+public class BroadcastExchangePOperator extends AbstractExchangePOperator {
private INodeDomain domain;
- public BroadcastPOperator(INodeDomain domain) {
+ public BroadcastExchangePOperator(INodeDomain domain) {
this.domain = domain;
}
@@ -67,7 +67,7 @@
@Override
public Pair<IConnectorDescriptor, TargetConstraint> createConnectorDescriptor(IConnectorDescriptorRegistry spec,
ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context) throws AlgebricksException {
- IConnectorDescriptor conn = new MToNReplicatingConnectorDescriptor(spec);
+ IConnectorDescriptor conn = new MToNBroadcastConnectorDescriptor(spec);
return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
}
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
new file mode 100644
index 0000000..65715b7
--- /dev/null
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import org.apache.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderedPartitionedProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.IRangePartitionType.RangePartitioningType;
+import org.apache.hyracks.dataflow.std.join.IMergeJoinCheckerFactory;
+import org.apache.hyracks.dataflow.std.join.MergeJoinOperatorDescriptor;
+
+public class MergeJoinPOperator extends AbstractJoinPOperator {
+
+ private final int memSize;
+ protected final List<LogicalVariable> keysLeftBranch;
+ protected final List<LogicalVariable> keysRightBranch;
+ private final IBinaryComparatorFactoryProvider bcfp;
+ private final IMergeJoinCheckerFactory mjcf;
+ private IRangeMap rangeMap;
+
+ public MergeJoinPOperator(JoinKind kind, JoinPartitioningType partitioningType, int memSize,
+ List<LogicalVariable> sideLeft, List<LogicalVariable> sideRight, IBinaryComparatorFactoryProvider bcfp,
+ IMergeJoinCheckerFactory mjcf, IRangeMap rangeMap) {
+ super(kind, partitioningType);
+ this.memSize = memSize;
+ this.keysLeftBranch = sideLeft;
+ this.keysRightBranch = sideRight;
+ this.bcfp = bcfp;
+ this.mjcf = mjcf;
+ this.rangeMap = rangeMap;
+ }
+
+ @Override
+ public PhysicalOperatorTag getOperatorTag() {
+ return PhysicalOperatorTag.SORT_MERGE_INTERVAL_JOIN;
+ }
+
+ @Override
+ public boolean isMicroOperator() {
+ return false;
+ }
+
+ @Override
+ public void computeDeliveredProperties(ILogicalOperator iop, IOptimizationContext context) {
+ IPartitioningProperty pp = null;
+ ArrayList<OrderColumn> order = new ArrayList<OrderColumn>();
+ for (LogicalVariable v : keysLeftBranch) {
+ order.add(new OrderColumn(v, OrderKind.ASC));
+ }
+ pp = new OrderedPartitionedProperty(order, null, rangeMap, RangePartitioningType.PROJECT);
+ List<ILocalStructuralProperty> propsLocal = new ArrayList<ILocalStructuralProperty>();
+ propsLocal.add(new LocalOrderProperty(order));
+ deliveredProperties = new StructuralPropertiesVector(pp, propsLocal);
+ }
+
+ @Override
+ public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator iop,
+ IPhysicalPropertiesVector reqdByParent) {
+ StructuralPropertiesVector[] pv = new StructuralPropertiesVector[2];
+ AbstractLogicalOperator op = (AbstractLogicalOperator) iop;
+
+ IPartitioningProperty ppLeft = null;
+ List<ILocalStructuralProperty> ispLeft = new ArrayList<ILocalStructuralProperty>();
+ IPartitioningProperty ppRight = null;
+ List<ILocalStructuralProperty> ispRight = new ArrayList<ILocalStructuralProperty>();
+ if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED) {
+ ArrayList<OrderColumn> orderLeft = new ArrayList<OrderColumn>();
+ for (LogicalVariable v : keysLeftBranch) {
+ orderLeft.add(new OrderColumn(v, OrderKind.ASC));
+ }
+ ppLeft = new OrderedPartitionedProperty(orderLeft, null, rangeMap, RangePartitioningType.PROJECT);
+ ispLeft.add(new LocalOrderProperty(orderLeft));
+
+ ArrayList<OrderColumn> orderRight = new ArrayList<OrderColumn>();
+ for (LogicalVariable v : keysRightBranch) {
+ orderRight.add(new OrderColumn(v, OrderKind.ASC));
+ }
+ ppRight = new OrderedPartitionedProperty(orderRight, null, rangeMap, RangePartitioningType.SPLIT);
+ ispRight.add(new LocalOrderProperty(orderRight));
+ }
+
+ pv[0] = new StructuralPropertiesVector(ppLeft, ispLeft);
+ pv[1] = new StructuralPropertiesVector(ppRight, ispRight);
+ IPartitioningRequirementsCoordinator prc = IPartitioningRequirementsCoordinator.NO_COORDINATION;
+ return new PhysicalRequirements(pv, prc);
+ }
+
+ @Override
+ public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+ IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+ throws AlgebricksException {
+ int[] keysLeft = JobGenHelper.variablesToFieldIndexes(keysLeftBranch, inputSchemas[0]);
+ int[] keysRight = JobGenHelper.variablesToFieldIndexes(keysRightBranch, inputSchemas[1]);
+ IVariableTypeEnvironment env = context.getTypeEnvironment(op);
+ IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[keysLeft.length];
+ int i = 0;
+ for (LogicalVariable v : keysLeftBranch) {
+ Object t = env.getVarType(v);
+ comparatorFactories[i++] = bcfp.getBinaryComparatorFactory(t, true);
+ }
+
+ IOperatorDescriptorRegistry spec = builder.getJobSpec();
+ RecordDescriptor recordDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), opSchema,
+ context);
+
+ MergeJoinOperatorDescriptor opDesc = new MergeJoinOperatorDescriptor(spec, memSize, recordDescriptor, keysLeft,
+ keysRight, comparatorFactories, mjcf);
+ contributeOpDesc(builder, (AbstractLogicalOperator) op, opDesc);
+
+ ILogicalOperator src1 = op.getInputs().get(0).getValue();
+ builder.contributeGraphEdge(src1, 0, op, 0);
+ ILogicalOperator src2 = op.getInputs().get(1).getValue();
+ builder.contributeGraphEdge(src2, 0, op, 1);
+ }
+}
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
index bfe8b36..66f9d93 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
@@ -34,7 +34,7 @@
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
-import org.apache.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.connectors.MToNBroadcastConnectorDescriptor;
public class RandomMergeExchangePOperator extends AbstractExchangePOperator {
@@ -58,7 +58,7 @@
@Override
public Pair<IConnectorDescriptor, TargetConstraint> createConnectorDescriptor(IConnectorDescriptorRegistry spec,
ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context) {
- IConnectorDescriptor conn = new MToNReplicatingConnectorDescriptor(spec);
+ IConnectorDescriptor conn = new MToNBroadcastConnectorDescriptor(spec);
return new Pair<IConnectorDescriptor, TargetConstraint>(conn, TargetConstraint.ONE);
}
}
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionPOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
similarity index 96%
rename from algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionPOperator.java
rename to algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
index 42e6bcf..9c53755c 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
@@ -39,11 +39,11 @@
import org.apache.hyracks.dataflow.common.data.partition.RandomPartitionComputerFactory;
import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
-public class RandomPartitionPOperator extends AbstractExchangePOperator {
+public class RandomPartitionExchangePOperator extends AbstractExchangePOperator {
private final INodeDomain domain;
- public RandomPartitionPOperator(INodeDomain domain) {
+ public RandomPartitionExchangePOperator(INodeDomain domain) {
this.domain = domain;
}
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
similarity index 80%
rename from algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java
rename to algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
index dca3d97..d99a34a 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
@@ -19,7 +19,6 @@
package org.apache.hyracks.algebricks.core.algebra.operators.physical;
import java.util.ArrayList;
-import java.util.LinkedList;
import java.util.List;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -30,9 +29,11 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty.PropertyType;
import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
@@ -50,18 +51,22 @@
import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
import org.apache.hyracks.dataflow.common.data.partition.range.FieldRangePartitionComputerFactory;
import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.IRangePartitionType.RangePartitioningType;
import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
-public class RangePartitionPOperator extends AbstractExchangePOperator {
+public class RangePartitionExchangePOperator extends AbstractExchangePOperator {
private List<OrderColumn> partitioningFields;
private INodeDomain domain;
private IRangeMap rangeMap;
+ private RangePartitioningType rangeType;
- public RangePartitionPOperator(List<OrderColumn> partitioningFields, INodeDomain domain, IRangeMap rangeMap) {
+ public RangePartitionExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain, IRangeMap rangeMap,
+ RangePartitioningType rangeType) {
this.partitioningFields = partitioningFields;
this.domain = domain;
this.rangeMap = rangeMap;
+ this.rangeType = rangeType;
}
@Override
@@ -79,8 +84,19 @@
@Override
public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
- IPartitioningProperty p = new OrderedPartitionedProperty(new ArrayList<OrderColumn>(partitioningFields), domain);
- this.deliveredProperties = new StructuralPropertiesVector(p, new LinkedList<ILocalStructuralProperty>());
+ IPartitioningProperty p = new OrderedPartitionedProperty(new ArrayList<OrderColumn>(partitioningFields), domain,
+ rangeMap, rangeType);
+ AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+ List<ILocalStructuralProperty> op2Locals = op2.getDeliveredPhysicalProperties().getLocalProperties();
+ List<ILocalStructuralProperty> locals = new ArrayList<ILocalStructuralProperty>();
+ for (ILocalStructuralProperty prop : op2Locals) {
+ if (prop.getPropertyType() == PropertyType.LOCAL_ORDER_PROPERTY) {
+ locals.add(prop);
+ } else {
+ break;
+ }
+ }
+ this.deliveredProperties = new StructuralPropertiesVector(p, locals);
}
@Override
@@ -113,14 +129,15 @@
comps[i] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
i++;
}
- ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(sortFields, comps, rangeMap);
+ ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(sortFields, comps, rangeMap,
+ rangeType);
IConnectorDescriptor conn = new MToNPartitioningConnectorDescriptor(spec, tpcf);
return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
}
@Override
public String toString() {
- return getOperatorTag().toString() + " " + partitioningFields + " SPLIT COUNT:" + rangeMap.getSplitCount();
+ return getOperatorTag().toString() + " " + partitioningFields + " " + rangeType;
}
}
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergePOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
similarity index 88%
rename from algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergePOperator.java
rename to algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
index d47c31f..9068084 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
@@ -23,7 +23,6 @@
import java.util.List;
import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.ListSet;
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder.TargetConstraint;
import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
@@ -42,9 +41,9 @@
import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
import org.apache.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderedPartitionedProperty;
import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
-import org.apache.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
@@ -55,18 +54,22 @@
import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
import org.apache.hyracks.dataflow.common.data.partition.range.FieldRangePartitionComputerFactory;
import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.IRangePartitionType.RangePartitioningType;
import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
-public class RangePartitionMergePOperator extends AbstractExchangePOperator {
+public class RangePartitionMergeExchangePOperator extends AbstractExchangePOperator {
private List<OrderColumn> partitioningFields;
private INodeDomain domain;
private IRangeMap rangeMap;
+ private RangePartitioningType rangeType;
- public RangePartitionMergePOperator(List<OrderColumn> partitioningFields, INodeDomain domain, IRangeMap rangeMap) {
+ public RangePartitionMergeExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain, IRangeMap rangeMap,
+ RangePartitioningType rangeType) {
this.partitioningFields = partitioningFields;
this.domain = domain;
this.rangeMap = rangeMap;
+ this.rangeType = rangeType;
}
@Override
@@ -84,11 +87,7 @@
@Override
public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
- List<LogicalVariable> varList = new ArrayList<LogicalVariable>();
- for (OrderColumn oc : partitioningFields) {
- varList.add(oc.getColumn());
- }
- IPartitioningProperty p = new UnorderedPartitionedProperty(new ListSet<LogicalVariable>(varList), domain);
+ IPartitioningProperty p = new OrderedPartitionedProperty(partitioningFields, domain, rangeMap, rangeType);
AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
List<ILocalStructuralProperty> op2Locals = op2.getDeliveredPhysicalProperties().getLocalProperties();
List<ILocalStructuralProperty> locals = new ArrayList<ILocalStructuralProperty>();
@@ -99,7 +98,6 @@
break;
}
}
-
this.deliveredProperties = new StructuralPropertiesVector(p, locals);
}
@@ -113,7 +111,8 @@
columns.add(new OrderColumn(var, oc.getOrder()));
}
orderProps.add(new LocalOrderProperty(columns));
- StructuralPropertiesVector[] r = new StructuralPropertiesVector[] { new StructuralPropertiesVector(null,
+ OrderedPartitionedProperty orderedPP = new OrderedPartitionedProperty(partitioningFields, domain);
+ StructuralPropertiesVector[] r = new StructuralPropertiesVector[] { new StructuralPropertiesVector(orderedPP,
orderProps) };
return new PhysicalRequirements(r, IPartitioningRequirementsCoordinator.NO_COORDINATION);
}
@@ -142,14 +141,15 @@
comps[i] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
i++;
}
- ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(sortFields, comps, rangeMap);
+ ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(sortFields, comps, rangeMap,
+ rangeType);
IConnectorDescriptor conn = new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, sortFields, comps, nkcf);
return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
}
@Override
public String toString() {
- return getOperatorTag().toString() + " " + partitioningFields + " SPLIT COUNT:" + rangeMap.getSplitCount();
+ return getOperatorTag().toString() + " " + partitioningFields + " " + rangeType;
}
}
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
index f264284..78f454e 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
@@ -124,8 +124,8 @@
IPhysicalPropertiesVector reqdByParent) {
List<ILocalStructuralProperty> localProps = new ArrayList<ILocalStructuralProperty>(sortColumns.length);
localProps.add(new LocalOrderProperty(Arrays.asList(sortColumns)));
- StructuralPropertiesVector[] r = new StructuralPropertiesVector[] { new StructuralPropertiesVector(null,
- localProps) };
+ StructuralPropertiesVector[] r = new StructuralPropertiesVector[] {
+ new StructuralPropertiesVector(null, localProps) };
return new PhysicalRequirements(r, IPartitioningRequirementsCoordinator.NO_COORDINATION);
}
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
index 3142d10..95e3ee1 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
@@ -26,8 +26,31 @@
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
public interface IPartitioningProperty extends IStructuralProperty {
+ /**
+ * The Partitioning Types define the method data is transfered between partitions and/or properties of the data.
+ */
public enum PartitioningType {
- UNPARTITIONED, RANDOM, BROADCAST, UNORDERED_PARTITIONED, ORDERED_PARTITIONED
+ /**
+ * Data is not partitioned.
+ */
+ UNPARTITIONED,
+ /**
+ * Data is partitioned without a repeatable method.
+ */
+ RANDOM,
+ /**
+ * Data is replicated to all partitions.
+ */
+ BROADCAST,
+ /**
+ * Data is hash partitioned.
+ */
+ UNORDERED_PARTITIONED,
+ /**
+ * Data is range partitioned (only used on data that has a total order).
+ * The partitions are order based on the data range.
+ */
+ ORDERED_PARTITIONED
}
static final INodeDomain DOMAIN_FOR_UNPARTITIONED_DATA = new INodeDomain() {
@@ -50,7 +73,8 @@
}
@Override
- public void normalize(Map<LogicalVariable, EquivalenceClass> equivalenceClasses, List<FunctionalDependency> fds) {
+ public void normalize(Map<LogicalVariable, EquivalenceClass> equivalenceClasses,
+ List<FunctionalDependency> fds) {
// do nothing
}
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
index f28bc56..41fb5a4 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
@@ -25,22 +25,43 @@
import org.apache.hyracks.algebricks.core.algebra.base.EquivalenceClass;
import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.IRangePartitionType.RangePartitioningType;
public class OrderedPartitionedProperty implements IPartitioningProperty {
- private ArrayList<OrderColumn> orderColumns;
+ private List<OrderColumn> orderColumns;
private INodeDomain domain;
+ private IRangeMap rangeMap;
+ private RangePartitioningType rangeType;
- public OrderedPartitionedProperty(ArrayList<OrderColumn> orderColumns, INodeDomain domain) {
+ public OrderedPartitionedProperty(List<OrderColumn> orderColumns, INodeDomain domain, IRangeMap rangeMap,
+ RangePartitioningType rangeType) {
this.domain = domain;
this.orderColumns = orderColumns;
+ this.rangeMap = rangeMap;
+ this.rangeType = rangeType;
}
- public ArrayList<OrderColumn> getOrderColumns() {
+ public OrderedPartitionedProperty(List<OrderColumn> orderColumns, INodeDomain domain, IRangeMap rangeMap) {
+ this.domain = domain;
+ this.orderColumns = orderColumns;
+ this.rangeMap = rangeMap;
+ this.rangeType = rangeType;
+ }
+
+ public OrderedPartitionedProperty(List<OrderColumn> orderColumns, INodeDomain domain, IRangeMap rangeMap) {
+ this.domain = domain;
+ this.orderColumns = orderColumns;
+ this.rangeMap = rangeMap;
+ this.rangeType = RangePartitioningType.PROJECT;
+ }
+
+ public List<OrderColumn> getOrderColumns() {
return orderColumns;
}
- public ArrayList<LogicalVariable> getColumns() {
+ public List<LogicalVariable> getColumns() {
ArrayList<LogicalVariable> cols = new ArrayList<LogicalVariable>(orderColumns.size());
for (OrderColumn oc : orderColumns) {
cols.add(oc.getColumn());
@@ -53,9 +74,13 @@
return PartitioningType.ORDERED_PARTITIONED;
}
+ public RangePartitioningType getRangePartitioningType() {
+ return rangeType;
+ }
+
@Override
public String toString() {
- return getPartitioningType().toString() + orderColumns;
+ return getPartitioningType().toString() + " Column(s): " + orderColumns + " Range Type: " + rangeType;
}
@Override
@@ -71,6 +96,10 @@
}
}
+ public IRangeMap getRangeMap() {
+ return rangeMap;
+ }
+
@Override
public INodeDomain getNodeDomain() {
return domain;
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
index ae9f4f1..f167a56 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
@@ -55,9 +55,8 @@
return k;
}
- public static boolean matchLocalProperties(List<ILocalStructuralProperty> reqd,
- List<ILocalStructuralProperty> dlvd, Map<LogicalVariable, EquivalenceClass> equivalenceClasses,
- List<FunctionalDependency> fds) {
+ public static boolean matchLocalProperties(List<ILocalStructuralProperty> reqd, List<ILocalStructuralProperty> dlvd,
+ Map<LogicalVariable, EquivalenceClass> equivalenceClasses, List<FunctionalDependency> fds) {
if (reqd == null) {
return true;
}
@@ -177,7 +176,7 @@
/**
* Converts a list of OrderColumns to a list of LogicalVariables.
- *
+ *
* @param orderColumns
* , a list of OrderColumns
* @return the list of LogicalVariables
@@ -208,7 +207,7 @@
return true;
}
- public static ArrayList<OrderColumn> applyFDsToOrderColumns(ArrayList<OrderColumn> orderColumns,
+ public static List<OrderColumn> applyFDsToOrderColumns(List<OrderColumn> orderColumns,
List<FunctionalDependency> fds) {
// the set of vars. is ordered
// so we try the variables in order from last to first
@@ -235,7 +234,7 @@
return norm;
}
- public static ArrayList<OrderColumn> replaceOrderColumnsByEqClasses(ArrayList<OrderColumn> orderColumns,
+ public static List<OrderColumn> replaceOrderColumnsByEqClasses(List<OrderColumn> orderColumns,
Map<LogicalVariable, EquivalenceClass> equivalenceClasses) {
if (equivalenceClasses == null || equivalenceClasses.isEmpty()) {
return orderColumns;
@@ -256,7 +255,7 @@
return norm;
}
- private static boolean impliedByPrefix(ArrayList<OrderColumn> vars, int i, FunctionalDependency fdep) {
+ private static boolean impliedByPrefix(List<OrderColumn> vars, int i, FunctionalDependency fdep) {
if (!fdep.getTail().contains(vars.get(i).getColumn())) {
return false;
}
diff --git a/algebricks/algebricks-examples/piglet-example/pom.xml b/algebricks/algebricks-examples/piglet-example/pom.xml
index a037db5..f9129d7 100644
--- a/algebricks/algebricks-examples/piglet-example/pom.xml
+++ b/algebricks/algebricks-examples/piglet-example/pom.xml
@@ -48,6 +48,7 @@
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>
+ <version>1.2</version>
<executions>
<execution>
<id>add-source</id>
diff --git a/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java b/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
index 165530a..2d1a4c9 100644
--- a/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
@@ -37,10 +37,8 @@
/**
* Matches the following operator pattern:
* (select) <-- ((assign)* <-- (select)*)+
- *
* Consolidates the selects to:
* (select) <-- (assign)*
- *
*/
public class ConsolidateSelectsRule implements IAlgebraicRewriteRule {
@@ -51,7 +49,7 @@
@Override
public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
- AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
return false;
}
@@ -59,30 +57,30 @@
IFunctionInfo andFn = context.getMetadataProvider().lookupFunction(AlgebricksBuiltinFunctions.AND);
// New conjuncts for consolidated select.
- AbstractFunctionCallExpression conj = null;
+ AbstractFunctionCallExpression conj = null;
AbstractLogicalOperator topMostOp = null;
AbstractLogicalOperator selectParent = null;
AbstractLogicalOperator nextSelect = firstSelect;
- do {
- // Skip through assigns.
+ do {
+ // Skip through assigns.
do {
- selectParent = nextSelect;
- nextSelect = (AbstractLogicalOperator) selectParent.getInputs().get(0).getValue();
+ selectParent = nextSelect;
+ nextSelect = (AbstractLogicalOperator) selectParent.getInputs().get(0).getValue();
} while (nextSelect.getOperatorTag() == LogicalOperatorTag.ASSIGN);
// Stop if the child op is not a select.
if (nextSelect.getOperatorTag() != LogicalOperatorTag.SELECT) {
- break;
- }
+ break;
+ }
// Remember the top-most op that we are not removing.
topMostOp = selectParent;
-
+
// Initialize the new conjuncts, if necessary.
if (conj == null) {
- conj = new ScalarFunctionCallExpression(andFn);
- // Add the first select's condition.
- conj.getArguments().add(new MutableObject<ILogicalExpression>(firstSelect.getCondition().getValue()));
+ conj = new ScalarFunctionCallExpression(andFn);
+ // Add the first select's condition.
+ conj.getArguments().add(new MutableObject<ILogicalExpression>(firstSelect.getCondition().getValue()));
}
-
+
// Consolidate all following selects.
do {
// Add the condition nextSelect to the new list of conjuncts.
@@ -90,21 +88,21 @@
selectParent = nextSelect;
nextSelect = (AbstractLogicalOperator) nextSelect.getInputs().get(0).getValue();
} while (nextSelect.getOperatorTag() == LogicalOperatorTag.SELECT);
-
+
// Hook up the input of the top-most remaining op if necessary.
if (topMostOp.getOperatorTag() == LogicalOperatorTag.ASSIGN || topMostOp == firstSelect) {
- topMostOp.getInputs().set(0, selectParent.getInputs().get(0));
+ topMostOp.getInputs().set(0, selectParent.getInputs().get(0));
}
-
+
// Prepare for next iteration.
nextSelect = selectParent;
} while (true);
-
- // Did we consolidate any selects?
+
+ // Did we consolidate any selects?
if (conj == null) {
- return false;
+ return false;
}
-
+
// Set the new conjuncts.
firstSelect.getCondition().setValue(conj);
context.computeAndSetTypeEnvironmentForOperator(firstSelect);
diff --git a/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java b/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
index 2181efa..03a5e7f 100644
--- a/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
@@ -51,7 +51,7 @@
import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.FDsAndEquivClassesVisitor;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractStableSortPOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.physical.BroadcastPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.BroadcastExchangePOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.ExternalGroupByPOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.HashPartitionExchangePOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.HashPartitionMergeExchangePOperator;
@@ -59,9 +59,9 @@
import org.apache.hyracks.algebricks.core.algebra.operators.physical.PreSortedDistinctByPOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.PreclusteredGroupByPOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.RandomMergeExchangePOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.physical.RandomPartitionPOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangePartitionMergePOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangePartitionPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.RandomPartitionExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangePartitionExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangePartitionMergeExchangePOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.SortMergeExchangePOperator;
import org.apache.hyracks.algebricks.core.algebra.operators.physical.StableSortPOperator;
import org.apache.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;
@@ -90,6 +90,7 @@
import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
import org.apache.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.IRangePartitionType.RangePartitioningType;
public class EnforceStructuralPropertiesRule implements IAlgebraicRewriteRule {
@@ -104,7 +105,8 @@
}
@Override
- public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+ throws AlgebricksException {
AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
// wait for the physical operators to be set first
if (op.getPhysicalOperator() == null) {
@@ -248,8 +250,8 @@
AbstractLogicalOperator child = (AbstractLogicalOperator) op.getInputs().get(childIndex).getValue();
IPhysicalPropertiesVector delivered = child.getDeliveredPhysicalProperties();
- AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Properties delivered by " + child.getPhysicalOperator()
- + ": " + delivered + "\n");
+ AlgebricksConfig.ALGEBRICKS_LOGGER
+ .finest(">>>> Properties delivered by " + child.getPhysicalOperator() + ": " + delivered + "\n");
IPartitioningRequirementsCoordinator prc = pr.getPartitioningCoordinator();
// Coordinates requirements by looking at the firstDeliveredPartitioning.
Pair<Boolean, IPartitioningProperty> pbpp = prc.coordinateRequirements(
@@ -308,8 +310,8 @@
if (opIsRedundantSort) {
if (AlgebricksConfig.DEBUG) {
- AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Removing redundant SORT operator "
- + op.getPhysicalOperator() + "\n");
+ AlgebricksConfig.ALGEBRICKS_LOGGER
+ .fine(">>>> Removing redundant SORT operator " + op.getPhysicalOperator() + "\n");
printOp(op);
}
changed = true;
@@ -332,7 +334,7 @@
private IPhysicalPropertiesVector newPropertiesDiff(AbstractLogicalOperator newChild,
IPhysicalPropertiesVector required, boolean mayExpandPartitioningProperties, IOptimizationContext context)
- throws AlgebricksException {
+ throws AlgebricksException {
IPhysicalPropertiesVector newDelivered = newChild.getDeliveredPhysicalProperties();
Map<LogicalVariable, EquivalenceClass> newChildEqClasses = context.getEquivalenceClassMap(newChild);
@@ -343,8 +345,8 @@
newChildEqClasses = context.getEquivalenceClassMap(newChild);
newChildFDs = context.getFDList(newChild);
}
- AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Required properties for new op. "
- + newChild.getPhysicalOperator() + ": " + required + "\n");
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(
+ ">>>> Required properties for new op. " + newChild.getPhysicalOperator() + ": " + required + "\n");
return newDelivered.getUnsatisfiedPropertiesFrom(required, mayExpandPartitioningProperties, newChildEqClasses,
newChildFDs);
@@ -418,16 +420,16 @@
IPhysicalPropertiesVector diffOfProperties, IOptimizationContext context) {
AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
if (op.getOperatorTag() != LogicalOperatorTag.ORDER
- || (op.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.STABLE_SORT && op
- .getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.IN_MEMORY_STABLE_SORT)
+ || (op.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.STABLE_SORT
+ && op.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.IN_MEMORY_STABLE_SORT)
|| delivered.getLocalProperties() == null) {
return false;
}
AbstractStableSortPOperator sortOp = (AbstractStableSortPOperator) op.getPhysicalOperator();
sortOp.computeLocalProperties(op);
ILocalStructuralProperty orderProp = sortOp.getOrderProperty();
- return PropertiesUtil.matchLocalProperties(Collections.singletonList(orderProp),
- delivered.getLocalProperties(), context.getEquivalenceClassMap(op), context.getFDList(op));
+ return PropertiesUtil.matchLocalProperties(Collections.singletonList(orderProp), delivered.getLocalProperties(),
+ context.getEquivalenceClassMap(op), context.getFDList(op));
}
private void addEnforcers(AbstractLogicalOperator op, int childIndex,
@@ -455,8 +457,8 @@
private void addLocalEnforcers(AbstractLogicalOperator op, int i, List<ILocalStructuralProperty> localProperties,
boolean nestedPlan, IOptimizationContext context) throws AlgebricksException {
if (AlgebricksConfig.DEBUG) {
- AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Adding local enforcers for local props = " + localProperties
- + "\n");
+ AlgebricksConfig.ALGEBRICKS_LOGGER
+ .fine(">>>> Adding local enforcers for local props = " + localProperties + "\n");
}
if (localProperties == null || localProperties.isEmpty()) {
@@ -475,8 +477,8 @@
}
case LOCAL_GROUPING_PROPERTY: {
LocalGroupingProperty g = (LocalGroupingProperty) prop;
- Collection<LogicalVariable> vars = (g.getPreferredOrderEnforcer() != null) ? g
- .getPreferredOrderEnforcer() : g.getColumnSet();
+ Collection<LogicalVariable> vars = (g.getPreferredOrderEnforcer() != null)
+ ? g.getPreferredOrderEnforcer() : g.getColumnSet();
List<OrderColumn> orderColumns = new ArrayList<OrderColumn>();
for (LogicalVariable v : vars) {
OrderColumn oc = new OrderColumn(v, OrderKind.ASC);
@@ -502,7 +504,7 @@
private Mutable<ILogicalOperator> enforceOrderProperties(List<LocalOrderProperty> oList,
Mutable<ILogicalOperator> topOp, boolean isMicroOp, IOptimizationContext context)
- throws AlgebricksException {
+ throws AlgebricksException {
List<Pair<IOrder, Mutable<ILogicalExpression>>> oe = new LinkedList<Pair<IOrder, Mutable<ILogicalExpression>>>();
for (LocalOrderProperty orderProperty : oList) {
for (OrderColumn oc : orderProperty.getOrderColumns()) {
@@ -539,9 +541,10 @@
pop = new RandomMergeExchangePOperator();
} else {
if (op.getAnnotations().containsKey(OperatorAnnotations.USE_RANGE_CONNECTOR)) {
- IRangeMap rangeMap = (IRangeMap) op.getAnnotations().get(
- OperatorAnnotations.USE_RANGE_CONNECTOR);
- pop = new RangePartitionMergePOperator(ordCols, domain, rangeMap);
+ IRangeMap rangeMap = (IRangeMap) op.getAnnotations()
+ .get(OperatorAnnotations.USE_RANGE_CONNECTOR);
+ pop = new RangePartitionMergeExchangePOperator(ordCols, domain, rangeMap,
+ RangePartitioningType.PROJECT);
} else {
OrderColumn[] sortColumns = new OrderColumn[ordCols.size()];
sortColumns = ordCols.toArray(sortColumns);
@@ -574,17 +577,37 @@
break;
}
case ORDERED_PARTITIONED: {
- pop = new RangePartitionPOperator(((OrderedPartitionedProperty) pp).getOrderColumns(), domain, null);
+ OrderedPartitionedProperty opp = (OrderedPartitionedProperty) pp;
+ List<ILocalStructuralProperty> cldLocals = deliveredByChild.getLocalProperties();
+ List<ILocalStructuralProperty> reqdLocals = required.getLocalProperties();
+ boolean propWasSet = false;
+ pop = null;
+ if (reqdLocals != null && cldLocals != null && allAreOrderProps(cldLocals)) {
+ AbstractLogicalOperator c = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
+ Map<LogicalVariable, EquivalenceClass> ecs = context.getEquivalenceClassMap(c);
+ List<FunctionalDependency> fds = context.getFDList(c);
+ if (PropertiesUtil.matchLocalProperties(reqdLocals, cldLocals, ecs, fds)) {
+ List<OrderColumn> orderColumns = getOrderColumnsFromGroupingProperties(reqdLocals,
+ cldLocals);
+ pop = new RangePartitionMergeExchangePOperator(orderColumns, domain, opp.getRangeMap(),
+ opp.getRangePartitioningType());
+ propWasSet = true;
+ }
+ }
+ if (!propWasSet) {
+ pop = new RangePartitionExchangePOperator(opp.getOrderColumns(), domain, opp.getRangeMap(),
+ opp.getRangePartitioningType());
+ }
break;
}
case BROADCAST: {
- pop = new BroadcastPOperator(domain);
+ pop = new BroadcastExchangePOperator(domain);
break;
}
case RANDOM: {
RandomPartitioningProperty rpp = (RandomPartitioningProperty) pp;
INodeDomain nd = rpp.getNodeDomain();
- pop = new RandomPartitionPOperator(nd);
+ pop = new RandomPartitionExchangePOperator(nd);
break;
}
default: {
@@ -600,8 +623,8 @@
OperatorPropertiesUtil.computeSchemaAndPropertiesRecIfNull(exchg, context);
context.computeAndSetTypeEnvironmentForOperator(exchg);
if (AlgebricksConfig.DEBUG) {
- AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Added partitioning enforcer "
- + exchg.getPhysicalOperator() + ".\n");
+ AlgebricksConfig.ALGEBRICKS_LOGGER
+ .fine(">>>> Added partitioning enforcer " + exchg.getPhysicalOperator() + ".\n");
printOp((AbstractLogicalOperator) op);
}
}
@@ -649,8 +672,8 @@
newOp.recomputeSchema();
newOp.computeDeliveredPhysicalProperties(context);
context.computeAndSetTypeEnvironmentForOperator(newOp);
- AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Structural properties for " + newOp.getPhysicalOperator()
- + ": " + newOp.getDeliveredPhysicalProperties() + "\n");
+ AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Structural properties for " + newOp.getPhysicalOperator() + ": "
+ + newOp.getDeliveredPhysicalProperties() + "\n");
PhysicalOptimizationsUtil.computeFDsAndEquivalenceClasses(newOp, context);
}
diff --git a/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java b/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java
index e1da07d..b3d53aa 100644
--- a/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java
+++ b/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java
@@ -18,9 +18,12 @@
*/
package org.apache.hyracks.api.dataflow.value;
+import java.util.List;
+
import org.apache.hyracks.api.comm.IFrameTupleAccessor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
public interface ITuplePartitionComputer {
- public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException;
+ public void partition(IFrameTupleAccessor accessor, int tIndex, int nParts, List<Integer> map)
+ throws HyracksDataException;
}
\ No newline at end of file
diff --git a/hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/ByteArrayPointableTest.java b/hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/ByteArrayPointableTest.java
index f58c8da..52d7ab4 100644
--- a/hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/ByteArrayPointableTest.java
+++ b/hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/ByteArrayPointableTest.java
@@ -19,18 +19,16 @@
package org.apache.hyracks.data.std.primitive;
+import static org.junit.Assert.assertTrue;
+
import org.junit.Test;
-
-import javax.xml.bind.DatatypeConverter;
-
-import static org.junit.Assert.*;
public class ByteArrayPointableTest {
- public static byte[] generatePointableBytes(byte[] bytes){
+ public static byte[] generatePointableBytes(byte[] bytes) {
byte[] ret = new byte[bytes.length + ByteArrayPointable.SIZE_OF_LENGTH];
- for (int i = 0; i < bytes.length; ++i){
- ret[i+ ByteArrayPointable.SIZE_OF_LENGTH] = bytes[i];
+ for (int i = 0; i < bytes.length; ++i) {
+ ret[i + ByteArrayPointable.SIZE_OF_LENGTH] = bytes[i];
}
ByteArrayPointable.putLength(bytes.length, ret, 0);
return ret;
@@ -38,32 +36,31 @@
@Test
public void testCompareTo() throws Exception {
- byte [] bytes = generatePointableBytes(new byte[] { 1, 2, 3, 4});
+ byte[] bytes = generatePointableBytes(new byte[] { 1, 2, 3, 4 });
ByteArrayPointable byteArrayPointable = new ByteArrayPointable();
byteArrayPointable.set(bytes, 0, bytes.length);
- testEqual(byteArrayPointable, generatePointableBytes(new byte[] { 1,2 ,3,4}));
+ testEqual(byteArrayPointable, generatePointableBytes(new byte[] { 1, 2, 3, 4 }));
- testLessThan(byteArrayPointable, generatePointableBytes(new byte[] {2}));
- testLessThan(byteArrayPointable, generatePointableBytes(new byte[] {1,2,3,5}));
- testLessThan(byteArrayPointable, generatePointableBytes(new byte[] {1,2,3,4,5}));
+ testLessThan(byteArrayPointable, generatePointableBytes(new byte[] { 2 }));
+ testLessThan(byteArrayPointable, generatePointableBytes(new byte[] { 1, 2, 3, 5 }));
+ testLessThan(byteArrayPointable, generatePointableBytes(new byte[] { 1, 2, 3, 4, 5 }));
- testGreaterThan(byteArrayPointable, generatePointableBytes(new byte[] { }));
- testGreaterThan(byteArrayPointable, generatePointableBytes(new byte[] { 0}));
- testGreaterThan(byteArrayPointable, generatePointableBytes(new byte[] { 1,2,3}));
+ testGreaterThan(byteArrayPointable, generatePointableBytes(new byte[] {}));
+ testGreaterThan(byteArrayPointable, generatePointableBytes(new byte[] { 0 }));
+ testGreaterThan(byteArrayPointable, generatePointableBytes(new byte[] { 1, 2, 3 }));
}
-
- void testEqual(ByteArrayPointable pointable, byte [] bytes){
+ void testEqual(ByteArrayPointable pointable, byte[] bytes) {
assertTrue(pointable.compareTo(bytes, 0, bytes.length) == 0);
}
- void testLessThan(ByteArrayPointable pointable, byte[] bytes){
+ void testLessThan(ByteArrayPointable pointable, byte[] bytes) {
assertTrue(pointable.compareTo(bytes, 0, bytes.length) < 0);
}
- void testGreaterThan(ByteArrayPointable pointable, byte[] bytes){
+ void testGreaterThan(ByteArrayPointable pointable, byte[] bytes) {
assertTrue(pointable.compareTo(bytes, 0, bytes.length) > 0);
}
}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTuplePairComparator.java b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTuplePairComparator.java
index e21a6ac..7066920 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTuplePairComparator.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTuplePairComparator.java
@@ -18,6 +18,7 @@
*/
package org.apache.hyracks.dataflow.common.comm.io;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -32,7 +33,7 @@
this.comparators = comparators;
}
- public int compare(FrameTupleAccessor accessor0, int tIndex0, FrameTupleAccessor accessor1, int tIndex1)
+ public int compare(IFrameTupleAccessor accessor0, int tIndex0, IFrameTupleAccessor accessor1, int tIndex1)
throws HyracksDataException {
int tStart0 = accessor0.getTupleStartOffset(tIndex0);
int fStartOffset0 = accessor0.getFieldSlotsLength() + tStart0;
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
index 14a7286..af252a5 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
@@ -18,6 +18,8 @@
*/
package org.apache.hyracks.dataflow.common.data.partition;
+import java.util.List;
+
import org.apache.hyracks.api.comm.IFrameTupleAccessor;
import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
@@ -43,9 +45,11 @@
}
return new ITuplePartitionComputer() {
@Override
- public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
+ public void partition(IFrameTupleAccessor accessor, int tIndex, int nParts, List<Integer> map)
+ throws HyracksDataException {
if (nParts == 1) {
- return 0;
+ map.add(0);
+ return;
}
int h = 0;
int startOffset = accessor.getTupleStartOffset(tIndex);
@@ -55,14 +59,14 @@
IBinaryHashFunction hashFn = hashFunctions[j];
int fStart = accessor.getFieldStartOffset(tIndex, fIdx);
int fEnd = accessor.getFieldEndOffset(tIndex, fIdx);
- int fh = hashFn
- .hash(accessor.getBuffer().array(), startOffset + slotLength + fStart, fEnd - fStart);
+ int fh = hashFn.hash(accessor.getBuffer().array(), startOffset + slotLength + fStart,
+ fEnd - fStart);
h = h * 31 + fh;
}
if (h < 0) {
h = -(h + 1);
}
- return h % nParts;
+ map.add(h % nParts);
}
};
}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java
index 458171c..4748c98 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java
@@ -18,6 +18,8 @@
*/
package org.apache.hyracks.dataflow.common.data.partition;
+import java.util.List;
+
import org.apache.hyracks.api.comm.IFrameTupleAccessor;
import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
@@ -30,7 +32,8 @@
private final int[] hashFields;
private final IBinaryHashFunctionFamily[] hashFunctionGeneratorFactories;
- public FieldHashPartitionComputerFamily(int[] hashFields, IBinaryHashFunctionFamily[] hashFunctionGeneratorFactories) {
+ public FieldHashPartitionComputerFamily(int[] hashFields,
+ IBinaryHashFunctionFamily[] hashFunctionGeneratorFactories) {
this.hashFields = hashFields;
this.hashFunctionGeneratorFactories = hashFunctionGeneratorFactories;
}
@@ -43,7 +46,8 @@
}
return new ITuplePartitionComputer() {
@Override
- public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
+ public void partition(IFrameTupleAccessor accessor, int tIndex, int nParts, List<Integer> map)
+ throws HyracksDataException {
int h = 0;
int startOffset = accessor.getTupleStartOffset(tIndex);
int slotLength = accessor.getFieldSlotsLength();
@@ -52,14 +56,14 @@
IBinaryHashFunction hashFn = hashFunctions[j];
int fStart = accessor.getFieldStartOffset(tIndex, fIdx);
int fEnd = accessor.getFieldEndOffset(tIndex, fIdx);
- int fh = hashFn
- .hash(accessor.getBuffer().array(), startOffset + slotLength + fStart, fEnd - fStart);
+ int fh = hashFn.hash(accessor.getBuffer().array(), startOffset + slotLength + fStart,
+ fEnd - fStart);
h += fh;
}
if (h < 0) {
h = -(h + 1);
}
- return h % nParts;
+ map.add(h % nParts);
}
};
}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
index 1cda8a2..1a71310 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
@@ -18,6 +18,7 @@
*/
package org.apache.hyracks.dataflow.common.data.partition;
+import java.util.List;
import java.util.Random;
import org.apache.hyracks.api.comm.IFrameTupleAccessor;
@@ -25,29 +26,28 @@
import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-public class RandomPartitionComputerFactory implements
- ITuplePartitionComputerFactory {
+public class RandomPartitionComputerFactory implements ITuplePartitionComputerFactory {
- private static final long serialVersionUID = 1L;
+ private static final long serialVersionUID = 1L;
- private final int domainCardinality;
+ private final int domainCardinality;
- public RandomPartitionComputerFactory(int domainCardinality) {
- this.domainCardinality = domainCardinality;
- }
+ public RandomPartitionComputerFactory(int domainCardinality) {
+ this.domainCardinality = domainCardinality;
+ }
- @Override
- public ITuplePartitionComputer createPartitioner() {
- return new ITuplePartitionComputer() {
+ @Override
+ public ITuplePartitionComputer createPartitioner() {
+ return new ITuplePartitionComputer() {
- private final Random random = new Random();
+ private final Random random = new Random();
- @Override
- public int partition(IFrameTupleAccessor accessor, int tIndex,
- int nParts) throws HyracksDataException {
- return random.nextInt(domainCardinality);
- }
- };
- }
+ @Override
+ public void partition(IFrameTupleAccessor accessor, int tIndex, int nParts, List<Integer> map)
+ throws HyracksDataException {
+ map.add(random.nextInt(domainCardinality));
+ }
+ };
+ }
}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java
index 14c5fa9..aeeed19 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java
@@ -18,6 +18,9 @@
*/
package org.apache.hyracks.dataflow.common.data.partition;
+import java.util.ArrayList;
+import java.util.List;
+
import org.apache.hyracks.api.comm.IFrameTupleAccessor;
import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
@@ -28,10 +31,12 @@
private int factor;
private ITuplePartitionComputerFactory delegateFactory;
+ private final ArrayList<Integer> repartitionMap;
public RepartitionComputerFactory(int factor, ITuplePartitionComputerFactory delegate) {
this.factor = factor;
this.delegateFactory = delegate;
+ this.repartitionMap = new ArrayList<Integer>();
}
@Override
@@ -40,8 +45,13 @@
private ITuplePartitionComputer delegate = delegateFactory.createPartitioner();
@Override
- public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
- return delegate.partition(accessor, tIndex, factor * nParts) / factor;
+ public void partition(IFrameTupleAccessor accessor, int tIndex, int nParts, List<Integer> map)
+ throws HyracksDataException {
+ delegate.partition(accessor, tIndex, factor * nParts, repartitionMap);
+ for (Integer h : repartitionMap) {
+ map.add(h / factor);
+ }
+ repartitionMap.clear();
}
};
}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFamily.java b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFamily.java
index 4ad6e5f..1668bb0 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFamily.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFamily.java
@@ -18,6 +18,9 @@
*/
package org.apache.hyracks.dataflow.common.data.partition;
+import java.util.ArrayList;
+import java.util.List;
+
import org.apache.hyracks.api.comm.IFrameTupleAccessor;
import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFamily;
@@ -27,12 +30,14 @@
private static final long serialVersionUID = 1L;
- private int factor;
- private ITuplePartitionComputerFamily delegateFactory;
+ private final int factor;
+ private final ITuplePartitionComputerFamily delegateFactory;
+ private final ArrayList<Integer> repartitionMap;
public RepartitionComputerFamily(int factor, ITuplePartitionComputerFamily delegate) {
this.factor = factor;
this.delegateFactory = delegate;
+ this.repartitionMap = new ArrayList<Integer>();
}
@Override
@@ -42,8 +47,13 @@
private ITuplePartitionComputer delegate = delegateFactory.createPartitioner(s);
@Override
- public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
- return delegate.partition(accessor, tIndex, factor * nParts) / factor;
+ public void partition(IFrameTupleAccessor accessor, int tIndex, int nParts, List<Integer> map)
+ throws HyracksDataException {
+ delegate.partition(accessor, tIndex, factor * nParts, repartitionMap);
+ for (Integer h : repartitionMap) {
+ map.add(h / factor);
+ }
+ repartitionMap.clear();
}
};
}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
index 06f92bb..b200889 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
@@ -18,40 +18,45 @@
*/
package org.apache.hyracks.dataflow.common.data.partition.range;
+import java.util.List;
+
import org.apache.hyracks.api.comm.IFrameTupleAccessor;
import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.partition.range.IRangePartitionType.RangePartitioningType;
public class FieldRangePartitionComputerFactory implements ITuplePartitionComputerFactory {
private static final long serialVersionUID = 1L;
private final int[] rangeFields;
private IRangeMap rangeMap;
private IBinaryComparatorFactory[] comparatorFactories;
+ private RangePartitioningType rangeType;
public FieldRangePartitionComputerFactory(int[] rangeFields, IBinaryComparatorFactory[] comparatorFactories,
- IRangeMap rangeMap) {
+ IRangeMap rangeMap, RangePartitioningType rangeType) {
this.rangeFields = rangeFields;
this.comparatorFactories = comparatorFactories;
this.rangeMap = rangeMap;
+ this.rangeType = rangeType;
}
- @Override
public ITuplePartitionComputer createPartitioner() {
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
}
return new ITuplePartitionComputer() {
- @Override
/**
- * Determine the range partition.
+ * Determine the range partition.
*/
- public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
+ public void partition(IFrameTupleAccessor accessor, int tIndex, int nParts, List<Integer> map)
+ throws HyracksDataException {
if (nParts == 1) {
- return 0;
+ map.add(0);
+ return;
}
int slotIndex = getRangePartition(accessor, tIndex);
// Map range partition to node partitions.
@@ -59,7 +64,8 @@
if (rangeMap.getSplitCount() + 1 > nParts) {
rangesPerPart = ((double) rangeMap.getSplitCount() + 1) / nParts;
}
- return (int) Math.floor(slotIndex / rangesPerPart);
+ map.add((int) Math.floor(slotIndex / rangesPerPart));
+ return;
}
/*
@@ -79,6 +85,7 @@
public int compareSlotAndFields(IFrameTupleAccessor accessor, int tIndex, int fieldIndex)
throws HyracksDataException {
+ // TODO convert to a binary search algorithm
int c = 0;
int startOffset = accessor.getTupleStartOffset(tIndex);
int slotLength = accessor.getFieldSlotsLength();
@@ -86,8 +93,8 @@
int fIdx = rangeFields[f];
int fStart = accessor.getFieldStartOffset(tIndex, fIdx);
int fEnd = accessor.getFieldEndOffset(tIndex, fIdx);
- c = comparators[f].compare(accessor.getBuffer().array(), startOffset + slotLength + fStart, fEnd
- - fStart, rangeMap.getByteArray(fieldIndex, f), rangeMap.getStartOffset(fieldIndex, f),
+ c = comparators[f].compare(accessor.getBuffer().array(), startOffset + slotLength + fStart,
+ fEnd - fStart, rangeMap.getByteArray(fieldIndex, f), rangeMap.getStartOffset(fieldIndex, f),
rangeMap.getLength(fieldIndex, f));
if (c != 0) {
return c;
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java
new file mode 100644
index 0000000..f387794
--- /dev/null
+++ b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.dataflow.common.data.partition.range;
+
+public interface IRangePartitionType {
+ public enum RangePartitioningType {
+ /**
+ * Partitioning is determined by finding the range partition where the first data point lies.
+ */
+ PROJECT,
+ /**
+ * Partitioning is determined by finding all the range partitions where the data has a point.
+ */
+ SPLIT,
+ /**
+ * Partitioning is determined by finding all the range partitions where the data has a point
+ * or comes after the data point.
+ */
+ REPLICATE
+ }
+}
diff --git a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
index 5e6c5eb..61273da 100644
--- a/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
+++ b/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
@@ -28,6 +28,7 @@
* The first split value for each field followed by the second split value for each field, etc.
*/
public class RangeMap implements IRangeMap, Serializable {
+ private static final long serialVersionUID = 1L;
private final int fields;
private final byte[] bytes;
private final int[] offsets;
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopHashTuplePartitionComputerFactory.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopHashTuplePartitionComputerFactory.java
index f78aea6..df2e2b5 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopHashTuplePartitionComputerFactory.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopHashTuplePartitionComputerFactory.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.dataflow.hadoop.data;
import java.io.DataInputStream;
+import java.util.List;
import org.apache.hadoop.io.Writable;
@@ -44,7 +45,8 @@
private final DataInputStream dis = new DataInputStream(bbis);
@Override
- public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
+ public void partition(IFrameTupleAccessor accessor, int tIndex, int nParts, List<Integer> map)
+ throws HyracksDataException {
int keyStart = accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength()
+ accessor.getFieldStartOffset(tIndex, 0);
bbis.setByteBuffer(accessor.getBuffer(), keyStart);
@@ -53,7 +55,7 @@
if (h < 0) {
h = -h;
}
- return h % nParts;
+ map.add(h % nParts);
}
};
}
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopNewPartitionerTuplePartitionComputerFactory.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopNewPartitionerTuplePartitionComputerFactory.java
index 39b388f..f75f953 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopNewPartitionerTuplePartitionComputerFactory.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopNewPartitionerTuplePartitionComputerFactory.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.dataflow.hadoop.data;
import java.io.DataInputStream;
+import java.util.List;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapreduce.Partitioner;
@@ -30,8 +31,8 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
-public class HadoopNewPartitionerTuplePartitionComputerFactory<K extends Writable, V extends Writable> extends
- AbstractClassBasedDelegate<Partitioner<K, V>> implements ITuplePartitionComputerFactory {
+public class HadoopNewPartitionerTuplePartitionComputerFactory<K extends Writable, V extends Writable>
+ extends AbstractClassBasedDelegate<Partitioner<K, V>>implements ITuplePartitionComputerFactory {
private static final long serialVersionUID = 1L;
private final ISerializerDeserializer<K> keyIO;
private final ISerializerDeserializer<V> valueIO;
@@ -50,7 +51,8 @@
private final DataInputStream dis = new DataInputStream(bbis);
@Override
- public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
+ public void partition(IFrameTupleAccessor accessor, int tIndex, int nParts, List<Integer> map)
+ throws HyracksDataException {
int keyStart = accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength()
+ accessor.getFieldStartOffset(tIndex, 0);
bbis.setByteBuffer(accessor.getBuffer(), keyStart);
@@ -59,7 +61,7 @@
+ accessor.getFieldStartOffset(tIndex, 1);
bbis.setByteBuffer(accessor.getBuffer(), valueStart);
V value = valueIO.deserialize(dis);
- return instance.getPartition(key, value, nParts);
+ map.add(instance.getPartition(key, value, nParts));
}
};
}
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopPartitionerTuplePartitionComputerFactory.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopPartitionerTuplePartitionComputerFactory.java
index 04a5491..aaac6ae 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopPartitionerTuplePartitionComputerFactory.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopPartitionerTuplePartitionComputerFactory.java
@@ -19,6 +19,7 @@
package org.apache.hyracks.dataflow.hadoop.data;
import java.io.DataInputStream;
+import java.util.List;
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.mapred.Partitioner;
@@ -30,8 +31,8 @@
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
-public class HadoopPartitionerTuplePartitionComputerFactory<K extends Writable, V extends Writable> extends
- AbstractClassBasedDelegate<Partitioner<K, V>> implements ITuplePartitionComputerFactory {
+public class HadoopPartitionerTuplePartitionComputerFactory<K extends Writable, V extends Writable>
+ extends AbstractClassBasedDelegate<Partitioner<K, V>>implements ITuplePartitionComputerFactory {
private static final long serialVersionUID = 1L;
private final ISerializerDeserializer<K> keyIO;
private final ISerializerDeserializer<V> valueIO;
@@ -50,7 +51,8 @@
private final DataInputStream dis = new DataInputStream(bbis);
@Override
- public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
+ public void partition(IFrameTupleAccessor accessor, int tIndex, int nParts, List<Integer> map)
+ throws HyracksDataException {
int keyStart = accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength()
+ accessor.getFieldStartOffset(tIndex, 0);
bbis.setByteBuffer(accessor.getBuffer(), keyStart);
@@ -59,7 +61,7 @@
+ accessor.getFieldStartOffset(tIndex, 1);
bbis.setByteBuffer(accessor.getBuffer(), valueStart);
V value = valueIO.deserialize(dis);
- return instance.getPartition(key, value, nParts);
+ map.add(instance.getPartition(key, value, nParts));
}
};
}
diff --git a/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/util/InputSplitsProxy.java b/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/util/InputSplitsProxy.java
index eed2885..28fd067 100644
--- a/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/util/InputSplitsProxy.java
+++ b/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/util/InputSplitsProxy.java
@@ -26,7 +26,6 @@
import java.io.Serializable;
import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapred.InputSplit;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.util.ReflectionUtils;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
new file mode 100644
index 0000000..41e6411
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.dataflow.std.connectors;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.IPartitionWriterFactory;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+
+public abstract class AbstractPartitionDataWriter implements IFrameWriter {
+ protected final int consumerPartitionCount;
+ protected final IFrameWriter[] pWriters;
+ protected final FrameTupleAppender[] appenders;
+ protected final FrameTupleAccessor tupleAccessor;
+ protected final IHyracksTaskContext ctx;
+ protected boolean allocatedFrame = false;
+
+ public AbstractPartitionDataWriter(IHyracksTaskContext ctx, int consumerPartitionCount,
+ IPartitionWriterFactory pwFactory, RecordDescriptor recordDescriptor) throws HyracksDataException {
+ this.consumerPartitionCount = consumerPartitionCount;
+ pWriters = new IFrameWriter[consumerPartitionCount];
+ appenders = new FrameTupleAppender[consumerPartitionCount];
+ for (int i = 0; i < consumerPartitionCount; ++i) {
+ try {
+ pWriters[i] = pwFactory.createFrameWriter(i);
+ appenders[i] = new FrameTupleAppender();
+ } catch (IOException e) {
+ throw new HyracksDataException(e);
+ }
+ }
+ tupleAccessor = new FrameTupleAccessor(recordDescriptor);
+ this.ctx = ctx;
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ for (int i = 0; i < pWriters.length; ++i) {
+ if (allocatedFrame) {
+ appenders[i].flush(pWriters[i], true);
+ }
+ pWriters[i].close();
+ }
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ for (int i = 0; i < pWriters.length; ++i) {
+ pWriters[i].open();
+ }
+ }
+
+ @Override
+ abstract public void nextFrame(ByteBuffer buffer) throws HyracksDataException;
+
+ protected void allocateFrames() throws HyracksDataException {
+ for (int i = 0; i < appenders.length; ++i) {
+ appenders[i].reset(new VSizeFrame(ctx), true);
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ for (int i = 0; i < appenders.length; ++i) {
+ pWriters[i].fail();
+ }
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
index 3e437e0..4f5fbe1 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
@@ -20,6 +20,7 @@
import java.io.IOException;
import java.nio.ByteBuffer;
+import java.util.ArrayList;
import org.apache.hyracks.api.comm.IFrameTupleAppender;
import org.apache.hyracks.api.comm.IFrameWriter;
@@ -39,6 +40,7 @@
private final IFrameTupleAppender[] appenders;
private final FrameTupleAccessor tupleAccessor;
private final ITuplePartitionComputer tpc;
+ private final ArrayList<Integer> map;
public LocalityAwarePartitionDataWriter(IHyracksTaskContext ctx, IPartitionWriterFactory pwFactory,
RecordDescriptor recordDescriptor, ITuplePartitionComputer tpc, int nConsumerPartitions,
@@ -57,11 +59,12 @@
}
tupleAccessor = new FrameTupleAccessor(recordDescriptor);
this.tpc = tpc;
+ map = new ArrayList<Integer>();
}
/*
* (non-Javadoc)
- *
+ *
* @see org.apache.hyracks.api.comm.IFrameWriter#open()
*/
@Override
@@ -73,7 +76,7 @@
/*
* (non-Javadoc)
- *
+ *
* @see
* org.apache.hyracks.api.comm.IFrameWriter#nextFrame(java.nio.ByteBuffer)
*/
@@ -82,14 +85,21 @@
tupleAccessor.reset(buffer);
int tupleCount = tupleAccessor.getTupleCount();
for (int i = 0; i < tupleCount; ++i) {
- int h = pWriters.length == 1 ? 0 : tpc.partition(tupleAccessor, i, pWriters.length);
- FrameUtils.appendToWriter(pWriters[h], appenders[h], tupleAccessor, i);
+ if (pWriters.length == 1) {
+ FrameUtils.appendToWriter(pWriters[0], appenders[0], tupleAccessor, i);
+ } else {
+ tpc.partition(tupleAccessor, i, pWriters.length, map);
+ for (Integer p : map) {
+ FrameUtils.appendToWriter(pWriters[p], appenders[p], tupleAccessor, i);
+ }
+ map.clear();
+ }
}
}
/*
* (non-Javadoc)
- *
+ *
* @see org.apache.hyracks.api.comm.IFrameWriter#fail()
*/
@Override
@@ -101,7 +111,7 @@
/*
* (non-Javadoc)
- *
+ *
* @see org.apache.hyracks.api.comm.IFrameWriter#close()
*/
@Override
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
new file mode 100644
index 0000000..d167543
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.dataflow.std.connectors;
+
+import java.nio.ByteBuffer;
+import java.util.BitSet;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.IPartitionCollector;
+import org.apache.hyracks.api.comm.IPartitionWriterFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.collectors.NonDeterministicChannelReader;
+import org.apache.hyracks.dataflow.std.collectors.NonDeterministicFrameReader;
+import org.apache.hyracks.dataflow.std.collectors.PartitionCollector;
+
+public class MToNBroadcastConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+ public MToNBroadcastConnectorDescriptor(IConnectorDescriptorRegistry spec) {
+ super(spec);
+ }
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+ IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+ throws HyracksDataException {
+ final IFrameWriter[] epWriters = new IFrameWriter[nConsumerPartitions];
+ for (int i = 0; i < nConsumerPartitions; ++i) {
+ epWriters[i] = edwFactory.createFrameWriter(i);
+ }
+ return new IFrameWriter() {
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ buffer.mark();
+ for (int i = 0; i < epWriters.length; ++i) {
+ if (i != 0) {
+ buffer.reset();
+ }
+ epWriters[i].nextFrame(buffer);
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ for (int i = 0; i < epWriters.length; ++i) {
+ epWriters[i].fail();
+ }
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ for (int i = 0; i < epWriters.length; ++i) {
+ epWriters[i].close();
+ }
+ }
+
+ @Override
+ public void open() throws HyracksDataException {
+ for (int i = 0; i < epWriters.length; ++i) {
+ epWriters[i].open();
+ }
+ }
+ };
+ }
+
+ @Override
+ public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc, int index,
+ int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+ BitSet expectedPartitions = new BitSet(nProducerPartitions);
+ expectedPartitions.set(0, nProducerPartitions);
+ NonDeterministicChannelReader channelReader = new NonDeterministicChannelReader(nProducerPartitions,
+ expectedPartitions);
+ NonDeterministicFrameReader frameReader = new NonDeterministicFrameReader(channelReader);
+ return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, frameReader, channelReader);
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
new file mode 100644
index 0000000..f4dc1f2
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.dataflow.std.connectors;
+
+import java.util.BitSet;
+
+import org.apache.hyracks.api.comm.IFrameReader;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.IPartitionCollector;
+import org.apache.hyracks.api.comm.IPartitionWriterFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.collectors.IPartitionBatchManager;
+import org.apache.hyracks.dataflow.std.collectors.NonDeterministicPartitionBatchManager;
+import org.apache.hyracks.dataflow.std.collectors.PartitionCollector;
+import org.apache.hyracks.dataflow.std.collectors.SortMergeFrameReader;
+
+public class MToNRangePartitionMergingConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ private final ITuplePartitionComputerFactory tprcf;
+ private final int[] sortFields;
+ private final IBinaryComparatorFactory[] comparatorFactories;
+ private final INormalizedKeyComputerFactory nkcFactory;
+ private final boolean stable;
+
+ public MToNRangePartitionMergingConnectorDescriptor(IConnectorDescriptorRegistry spec,
+ ITuplePartitionComputerFactory tprcf, int[] sortFields, IBinaryComparatorFactory[] comparatorFactories,
+ INormalizedKeyComputerFactory nkcFactory) {
+ this(spec, tprcf, sortFields, comparatorFactories, nkcFactory, false);
+ }
+
+ public MToNRangePartitionMergingConnectorDescriptor(IConnectorDescriptorRegistry spec,
+ ITuplePartitionComputerFactory tprcf, int[] sortFields, IBinaryComparatorFactory[] comparatorFactories,
+ INormalizedKeyComputerFactory nkcFactory, boolean stable) {
+ super(spec);
+ this.tprcf = tprcf;
+ this.sortFields = sortFields;
+ this.comparatorFactories = comparatorFactories;
+ this.nkcFactory = nkcFactory;
+ this.stable = stable;
+ }
+
+ @Override
+ public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+ IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+ throws HyracksDataException {
+ final PartitionRangeDataWriter rangeWriter = new PartitionRangeDataWriter(ctx, nConsumerPartitions, edwFactory,
+ recordDesc, tprcf.createPartitioner());
+ return rangeWriter;
+ }
+
+ @Override
+ public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc, int index,
+ int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+ IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+ for (int i = 0; i < comparatorFactories.length; ++i) {
+ comparators[i] = comparatorFactories[i].createBinaryComparator();
+ }
+ INormalizedKeyComputer nmkComputer = nkcFactory == null ? null : nkcFactory.createNormalizedKeyComputer();
+ IPartitionBatchManager pbm = new NonDeterministicPartitionBatchManager(nProducerPartitions);
+ IFrameReader sortMergeFrameReader = new SortMergeFrameReader(ctx, nProducerPartitions, nProducerPartitions,
+ sortFields, comparators, nmkComputer, recordDesc, pbm);
+ BitSet expectedPartitions = new BitSet();
+ expectedPartitions.set(0, nProducerPartitions);
+ return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, sortMergeFrameReader, pbm);
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
new file mode 100644
index 0000000..af66dff
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.dataflow.std.connectors;
+
+import java.util.BitSet;
+
+import org.apache.hyracks.api.comm.IFrameReader;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.IPartitionCollector;
+import org.apache.hyracks.api.comm.IPartitionWriterFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputer;
+import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
+import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.collectors.IPartitionBatchManager;
+import org.apache.hyracks.dataflow.std.collectors.NonDeterministicPartitionBatchManager;
+import org.apache.hyracks.dataflow.std.collectors.PartitionCollector;
+import org.apache.hyracks.dataflow.std.collectors.SortMergeFrameReader;
+
+public class MToNRangePartitioningConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+ private static final long serialVersionUID = 1L;
+
+ private final ITuplePartitionComputerFactory trpcf;
+ private final int[] sortFields;
+ private final IBinaryComparatorFactory[] comparatorFactories;
+ private final INormalizedKeyComputerFactory nkcFactory;
+ private final boolean stable;
+
+ public MToNRangePartitioningConnectorDescriptor(IConnectorDescriptorRegistry spec,
+ ITuplePartitionComputerFactory trpcf, int[] sortFields, IBinaryComparatorFactory[] comparatorFactories,
+ INormalizedKeyComputerFactory nkcFactory) {
+ this(spec, trpcf, sortFields, comparatorFactories, nkcFactory, false);
+ }
+
+ public MToNRangePartitioningConnectorDescriptor(IConnectorDescriptorRegistry spec,
+ ITuplePartitionComputerFactory trpcf, int[] sortFields, IBinaryComparatorFactory[] comparatorFactories,
+ INormalizedKeyComputerFactory nkcFactory, boolean stable) {
+ super(spec);
+ this.trpcf = trpcf;
+ this.sortFields = sortFields;
+ this.comparatorFactories = comparatorFactories;
+ this.nkcFactory = nkcFactory;
+ this.stable = stable;
+ }
+
+ @Override
+ public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+ IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+ throws HyracksDataException {
+ final PartitionRangeDataWriter rangeWriter = new PartitionRangeDataWriter(ctx, nConsumerPartitions, edwFactory,
+ recordDesc, trpcf.createPartitioner());
+ return rangeWriter;
+ }
+
+ @Override
+ public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc, int index,
+ int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+ IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+ for (int i = 0; i < comparatorFactories.length; ++i) {
+ comparators[i] = comparatorFactories[i].createBinaryComparator();
+ }
+ INormalizedKeyComputer nmkComputer = nkcFactory == null ? null : nkcFactory.createNormalizedKeyComputer();
+ IPartitionBatchManager pbm = new NonDeterministicPartitionBatchManager(nProducerPartitions);
+ IFrameReader sortMergeFrameReader = new SortMergeFrameReader(ctx, nProducerPartitions, nProducerPartitions,
+ sortFields, comparators, nmkComputer, recordDesc, pbm);
+ BitSet expectedPartitions = new BitSet();
+ expectedPartitions.set(0, nProducerPartitions);
+ return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, sortMergeFrameReader, pbm);
+ }
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
index 08df2c5..99f1557 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
@@ -18,62 +18,25 @@
*/
package org.apache.hyracks.dataflow.std.connectors;
-import java.io.IOException;
import java.nio.ByteBuffer;
+import java.util.ArrayList;
-import org.apache.hyracks.api.comm.IFrameWriter;
import org.apache.hyracks.api.comm.IPartitionWriterFactory;
-import org.apache.hyracks.api.comm.VSizeFrame;
import org.apache.hyracks.api.context.IHyracksTaskContext;
import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
-public class PartitionDataWriter implements IFrameWriter {
- private final int consumerPartitionCount;
- private final IFrameWriter[] pWriters;
- private final FrameTupleAppender[] appenders;
- private final FrameTupleAccessor tupleAccessor;
+public class PartitionDataWriter extends AbstractPartitionDataWriter {
private final ITuplePartitionComputer tpc;
- private final IHyracksTaskContext ctx;
- private boolean allocatedFrame = false;
+ private final ArrayList<Integer> map;
public PartitionDataWriter(IHyracksTaskContext ctx, int consumerPartitionCount, IPartitionWriterFactory pwFactory,
RecordDescriptor recordDescriptor, ITuplePartitionComputer tpc) throws HyracksDataException {
- this.consumerPartitionCount = consumerPartitionCount;
- pWriters = new IFrameWriter[consumerPartitionCount];
- appenders = new FrameTupleAppender[consumerPartitionCount];
- for (int i = 0; i < consumerPartitionCount; ++i) {
- try {
- pWriters[i] = pwFactory.createFrameWriter(i);
- appenders[i] = new FrameTupleAppender();
- } catch (IOException e) {
- throw new HyracksDataException(e);
- }
- }
- tupleAccessor = new FrameTupleAccessor(recordDescriptor);
+ super(ctx, consumerPartitionCount, pwFactory, recordDescriptor);
this.tpc = tpc;
- this.ctx = ctx;
- }
-
- @Override
- public void close() throws HyracksDataException {
- for (int i = 0; i < pWriters.length; ++i) {
- if (allocatedFrame) {
- appenders[i].flush(pWriters[i], true);
- }
- pWriters[i].close();
- }
- }
-
- @Override
- public void open() throws HyracksDataException {
- for (int i = 0; i < pWriters.length; ++i) {
- pWriters[i].open();
- }
+ this.map = new ArrayList<Integer>();
}
@Override
@@ -85,22 +48,11 @@
tupleAccessor.reset(buffer);
int tupleCount = tupleAccessor.getTupleCount();
for (int i = 0; i < tupleCount; ++i) {
- int h = tpc.partition(tupleAccessor, i, consumerPartitionCount);
- FrameUtils.appendToWriter(pWriters[h], appenders[h], tupleAccessor, i);
-
- }
- }
-
- private void allocateFrames() throws HyracksDataException {
- for (int i = 0; i < appenders.length; ++i) {
- appenders[i].reset(new VSizeFrame(ctx), true);
- }
- }
-
- @Override
- public void fail() throws HyracksDataException {
- for (int i = 0; i < appenders.length; ++i) {
- pWriters[i].fail();
+ tpc.partition(tupleAccessor, i, consumerPartitionCount, map);
+ for (Integer h : map) {
+ FrameUtils.appendToWriter(pWriters[h], appenders[h], tupleAccessor, i);
+ }
+ map.clear();
}
}
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
new file mode 100644
index 0000000..1c720fc
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.dataflow.std.connectors;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+
+import org.apache.hyracks.api.comm.IPartitionWriterFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+
+public class PartitionRangeDataWriter extends AbstractPartitionDataWriter {
+ private final ITuplePartitionComputer tpc;
+ private final ArrayList<Integer> map;
+
+ public PartitionRangeDataWriter(IHyracksTaskContext ctx, int consumerPartitionCount,
+ IPartitionWriterFactory pwFactory, RecordDescriptor recordDescriptor, ITuplePartitionComputer tpc)
+ throws HyracksDataException {
+ super(ctx, consumerPartitionCount, pwFactory, recordDescriptor);
+ this.tpc = tpc;
+ this.map = new ArrayList<Integer>();
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ if (!allocatedFrame) {
+ allocateFrames();
+ allocatedFrame = true;
+ }
+ tupleAccessor.reset(buffer);
+ int tupleCount = tupleAccessor.getTupleCount();
+ for (int i = 0; i < tupleCount; ++i) {
+ tpc.partition(tupleAccessor, i, consumerPartitionCount, map);
+ for (Integer h : map) {
+ FrameUtils.appendToWriter(pWriters[h], appenders[h], tupleAccessor, i);
+ }
+ map.clear();
+ }
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
index 052fe8c..912d466 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
@@ -30,7 +30,6 @@
import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputer;
import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -41,6 +40,7 @@
import org.apache.hyracks.dataflow.std.structures.ISerializableTable;
import org.apache.hyracks.dataflow.std.structures.SerializableHashTable;
import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+import org.apache.hyracks.dataflow.std.util.PartitionUtil;
public class HashSpillableTableFactory implements ISpillableTableFactory {
@@ -92,10 +92,10 @@
final FrameTuplePairComparator ftpcTuple = new FrameTuplePairComparator(storedKeys, storedKeys, comparators);
- final ITuplePartitionComputer tpc = tpcf.createPartitioner();
+ final PartitionUtil pu = new PartitionUtil(tpcf.createPartitioner(), tableSize);
- final INormalizedKeyComputer nkc = firstKeyNormalizerFactory == null ? null : firstKeyNormalizerFactory
- .createNormalizedKeyComputer();
+ final INormalizedKeyComputer nkc = firstKeyNormalizerFactory == null ? null
+ : firstKeyNormalizerFactory.createNormalizedKeyComputer();
int[] keyFieldsInPartialResults = new int[keyFields.length];
for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
@@ -160,8 +160,9 @@
int f0StartRel = storedKeysAccessor1.getFieldStartOffset(tIndex, sfIdx);
int f0EndRel = storedKeysAccessor1.getFieldEndOffset(tIndex, sfIdx);
int f0Start = f0StartRel + tStart + storedKeysAccessor1.getFieldSlotsLength();
- tPointers[ptr * 3 + 2] = nkc == null ? 0 : nkc.normalize(storedKeysAccessor1.getBuffer()
- .array(), f0Start, f0EndRel - f0StartRel);
+ tPointers[ptr * 3 + 2] = nkc == null ? 0
+ : nkc.normalize(storedKeysAccessor1.getBuffer().array(), f0Start,
+ f0EndRel - f0StartRel);
ptr++;
offset++;
} while (true);
@@ -186,7 +187,7 @@
public boolean insert(FrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
if (lastBufIndex < 0)
nextAvailableFrame();
- int entry = tpc.partition(accessor, tIndex, tableSize);
+ int entry = pu.hashPartitionKey(accessor, tIndex);
boolean foundGroup = false;
int offset = 0;
do {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
index 8a87486..4725911 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
@@ -27,7 +27,6 @@
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.IntegerPointable;
-import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import org.apache.hyracks.dataflow.std.group.AggregateState;
import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
@@ -50,7 +49,7 @@
/*
* (non-Javadoc)
- *
+ *
* @see org.apache.hyracks.dataflow.std.aggregations.
* IFieldAggregateDescriptorFactory
* #createAggregator(org.apache.hyracks.api.context.IHyracksTaskContext,
@@ -113,7 +112,8 @@
int count = 0;
int tupleOffset = accessor.getTupleStartOffset(tIndex);
int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
- sum += IntegerPointable.getInteger(accessor.getBuffer().array(), tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+ sum += IntegerPointable.getInteger(accessor.getBuffer().array(),
+ tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
count += 1;
if (!useObjectState) {
try {
@@ -139,7 +139,8 @@
int sum = 0, count = 0;
int tupleOffset = accessor.getTupleStartOffset(tIndex);
int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
- sum += IntegerPointable.getInteger(accessor.getBuffer().array(), tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+ sum += IntegerPointable.getInteger(accessor.getBuffer().array(),
+ tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
count += 1;
if (!useObjectState) {
ByteBuffer buf = ByteBuffer.wrap(data);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
index ae1c16f..92cb679 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
@@ -27,7 +27,6 @@
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.IntegerPointable;
-import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import org.apache.hyracks.dataflow.std.group.AggregateState;
import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
@@ -50,7 +49,7 @@
/*
* (non-Javadoc)
- *
+ *
* @see org.apache.hyracks.dataflow.std.aggregations.
* IFieldAggregateDescriptorFactory
* #createAggregator(org.apache.hyracks.api.context.IHyracksTaskContext,
@@ -118,7 +117,8 @@
int sum = 0, count = 0;
int tupleOffset = accessor.getTupleStartOffset(tIndex);
int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
- sum += IntegerPointable.getInteger(accessor.getBuffer().array(), tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+ sum += IntegerPointable.getInteger(accessor.getBuffer().array(),
+ tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
count += 1;
if (!useObjectState) {
ByteBuffer buf = ByteBuffer.wrap(data);
@@ -156,8 +156,10 @@
int count = 0;
int tupleOffset = accessor.getTupleStartOffset(tIndex);
int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
- sum += IntegerPointable.getInteger(accessor.getBuffer().array(), tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
- count += IntegerPointable.getInteger(accessor.getBuffer().array(), tupleOffset + accessor.getFieldSlotsLength() + fieldStart + 4);
+ sum += IntegerPointable.getInteger(accessor.getBuffer().array(),
+ tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+ count += IntegerPointable.getInteger(accessor.getBuffer().array(),
+ tupleOffset + accessor.getFieldSlotsLength() + fieldStart + 4);
if (!useObjectState) {
try {
fieldOutput.writeInt(sum);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
index 357a2f8..a1856c3 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
@@ -27,7 +27,6 @@
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.IntegerPointable;
-import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import org.apache.hyracks.dataflow.std.group.AggregateState;
import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
@@ -47,7 +46,7 @@
/*
* (non-Javadoc)
- *
+ *
* @see org.apache.hyracks.dataflow.std.aggregations.
* IFieldAggregateDescriptorFactory
* #createAggregator(org.apache.hyracks.api.context.IHyracksTaskContext,
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
index 4a83127..f0b8dca 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
@@ -27,7 +27,6 @@
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.FloatPointable;
-import org.apache.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer;
import org.apache.hyracks.dataflow.std.group.AggregateState;
import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
@@ -35,37 +34,35 @@
/**
*
*/
-public class FloatSumFieldAggregatorFactory implements
- IFieldAggregateDescriptorFactory {
+public class FloatSumFieldAggregatorFactory implements IFieldAggregateDescriptorFactory {
private static final long serialVersionUID = 1L;
private final int aggField;
private final boolean useObjectState;
-
- public FloatSumFieldAggregatorFactory(int aggField, boolean useObjState){
+
+ public FloatSumFieldAggregatorFactory(int aggField, boolean useObjState) {
this.aggField = aggField;
this.useObjectState = useObjState;
}
-
+
/* (non-Javadoc)
* @see org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory#createAggregator(org.apache.hyracks.api.context.IHyracksTaskContext, org.apache.hyracks.api.dataflow.value.RecordDescriptor, org.apache.hyracks.api.dataflow.value.RecordDescriptor)
*/
@Override
- public IFieldAggregateDescriptor createAggregator(IHyracksTaskContext ctx,
- RecordDescriptor inRecordDescriptor,
+ public IFieldAggregateDescriptor createAggregator(IHyracksTaskContext ctx, RecordDescriptor inRecordDescriptor,
RecordDescriptor outRecordDescriptor) throws HyracksDataException {
return new IFieldAggregateDescriptor() {
-
+
@Override
public void reset() {
-
+
}
-
+
@Override
- public void outputPartialResult(DataOutput fieldOutput, byte[] data,
- int offset, AggregateState state) throws HyracksDataException {
+ public void outputPartialResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+ throws HyracksDataException {
float sum;
if (!useObjectState) {
sum = FloatPointable.getFloat(data, offset);
@@ -78,10 +75,10 @@
throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
}
}
-
+
@Override
- public void outputFinalResult(DataOutput fieldOutput, byte[] data,
- int offset, AggregateState state) throws HyracksDataException {
+ public void outputFinalResult(DataOutput fieldOutput, byte[] data, int offset, AggregateState state)
+ throws HyracksDataException {
float sum;
if (!useObjectState) {
sum = FloatPointable.getFloat(data, offset);
@@ -94,26 +91,26 @@
throw new HyracksDataException("I/O exception when writing aggregation to the output buffer.");
}
}
-
+
@Override
public boolean needsObjectState() {
return useObjectState;
}
-
+
@Override
public boolean needsBinaryState() {
return !useObjectState;
}
-
+
@Override
- public void init(IFrameTupleAccessor accessor, int tIndex,
- DataOutput fieldOutput, AggregateState state)
+ public void init(IFrameTupleAccessor accessor, int tIndex, DataOutput fieldOutput, AggregateState state)
throws HyracksDataException {
float sum = 0;
int tupleOffset = accessor.getTupleStartOffset(tIndex);
int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
- sum += FloatPointable.getFloat(accessor.getBuffer().array(), tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+ sum += FloatPointable.getFloat(accessor.getBuffer().array(),
+ tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
if (!useObjectState) {
try {
@@ -125,26 +122,26 @@
state.state = sum;
}
}
-
+
@Override
public AggregateState createState() {
return new AggregateState(new Float(0.0));
}
-
+
@Override
public void close() {
// TODO Auto-generated method stub
-
+
}
-
+
@Override
- public void aggregate(IFrameTupleAccessor accessor, int tIndex,
- byte[] data, int offset, AggregateState state)
- throws HyracksDataException {
+ public void aggregate(IFrameTupleAccessor accessor, int tIndex, byte[] data, int offset,
+ AggregateState state) throws HyracksDataException {
float sum = 0;
int tupleOffset = accessor.getTupleStartOffset(tIndex);
int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
- sum += FloatPointable.getFloat(accessor.getBuffer().array(), tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+ sum += FloatPointable.getFloat(accessor.getBuffer().array(),
+ tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
if (!useObjectState) {
ByteBuffer buf = ByteBuffer.wrap(data);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
index 9983541..179297c 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
@@ -27,7 +27,6 @@
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.data.std.primitive.IntegerPointable;
-import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import org.apache.hyracks.dataflow.std.group.AggregateState;
import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptor;
import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory;
@@ -50,7 +49,7 @@
/*
* (non-Javadoc)
- *
+ *
* @see org.apache.hyracks.dataflow.std.aggregations.
* IFieldAggregateDescriptorFactory
* #createAggregator(org.apache.hyracks.api.context.IHyracksTaskContext,
@@ -107,7 +106,8 @@
int tupleOffset = accessor.getTupleStartOffset(tIndex);
int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
- sum += IntegerPointable.getInteger(accessor.getBuffer().array(), tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+ sum += IntegerPointable.getInteger(accessor.getBuffer().array(),
+ tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
if (!useObjectState) {
try {
@@ -143,7 +143,8 @@
int sum = 0;
int tupleOffset = accessor.getTupleStartOffset(tIndex);
int fieldStart = accessor.getFieldStartOffset(tIndex, aggField);
- sum += IntegerPointable.getInteger(accessor.getBuffer().array(), tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
+ sum += IntegerPointable.getInteger(accessor.getBuffer().array(),
+ tupleOffset + accessor.getFieldSlotsLength() + fieldStart);
if (!useObjectState) {
ByteBuffer buf = ByteBuffer.wrap(data);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/GroupingHashTable.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/GroupingHashTable.java
index 97cea25..2155c8a 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/GroupingHashTable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/GroupingHashTable.java
@@ -30,7 +30,6 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -42,6 +41,7 @@
import org.apache.hyracks.dataflow.std.group.AggregateState;
import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptor;
import org.apache.hyracks.dataflow.std.group.IAggregatorDescriptorFactory;
+import org.apache.hyracks.dataflow.std.util.PartitionUtil;
class GroupingHashTable {
/**
@@ -88,7 +88,7 @@
private final int[] keys;
private final IBinaryComparator[] comparators;
private final FrameTuplePairComparator ftpc;
- private final ITuplePartitionComputer tpc;
+ private final PartitionUtil pu;
private final IAggregatorDescriptor aggregator;
private final IFrame outputFrame;
@@ -101,7 +101,7 @@
GroupingHashTable(IHyracksTaskContext ctx, int[] fields, IBinaryComparatorFactory[] comparatorFactories,
ITuplePartitionComputerFactory tpcf, IAggregatorDescriptorFactory aggregatorFactory,
RecordDescriptor inRecordDescriptor, RecordDescriptor outRecordDescriptor, int tableSize)
- throws HyracksDataException {
+ throws HyracksDataException {
this.ctx = ctx;
buffers = new ArrayList<>();
@@ -121,7 +121,8 @@
comparators[i] = comparatorFactories[i].createBinaryComparator();
}
ftpc = new FrameTuplePairComparator(fields, storedKeys, comparators);
- tpc = tpcf.createPartitioner();
+
+ pu = new PartitionUtil(tpcf.createPartitioner(), tableSize);
int[] keyFieldsInPartialResults = new int[fields.length];
for (int i = 0; i < keyFieldsInPartialResults.length; i++) {
@@ -158,8 +159,8 @@
++lastBIndex;
}
- void insert(FrameTupleAccessor accessor, int tIndex) throws Exception {
- int entry = tpc.partition(accessor, tIndex, table.length);
+ void insert(FrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+ int entry = pu.hashPartitionKey(accessor, tIndex);
Link link = table[entry];
if (link == null) {
link = table[entry] = new Link();
@@ -190,8 +191,8 @@
aggregator.init(stateTupleBuilder, accessor, tIndex, newState);
- if (!appender.appendSkipEmptyField(stateTupleBuilder.getFieldEndOffsets(),
- stateTupleBuilder.getByteArray(), 0, stateTupleBuilder.getSize())) {
+ if (!appender.appendSkipEmptyField(stateTupleBuilder.getFieldEndOffsets(), stateTupleBuilder.getByteArray(),
+ 0, stateTupleBuilder.getSize())) {
addNewBuffer();
if (!appender.appendSkipEmptyField(stateTupleBuilder.getFieldEndOffsets(),
stateTupleBuilder.getByteArray(), 0, stateTupleBuilder.getSize())) {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
index e6e0afd..d2f77df 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
@@ -26,7 +26,6 @@
import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.io.FileReference;
@@ -35,15 +34,15 @@
import org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
import org.apache.hyracks.dataflow.common.io.RunFileWriter;
import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.util.PartitionUtil;
-class GraceHashJoinPartitionBuildOperatorNodePushable extends
- AbstractUnaryInputSinkOperatorNodePushable {
+class GraceHashJoinPartitionBuildOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
private final IHyracksTaskContext ctx;
private final Object stateId;
private final int numPartitions;
private final IBinaryComparator[] comparators;
private final FrameTupleAccessor accessor0;
- private final ITuplePartitionComputer hpc;
+ private final PartitionUtil pu;
private final FrameTupleAppender appender;
private IFrame[] outbufs;
private GraceHashJoinPartitionState state;
@@ -56,7 +55,8 @@
this.numPartitions = numPartitions;
accessor0 = new FrameTupleAccessor(inRecordDescriptor);
appender = new FrameTupleAppender();
- hpc = new FieldHashPartitionComputerFactory(keys, hashFunctionFactories).createPartitioner();
+ pu = new PartitionUtil(new FieldHashPartitionComputerFactory(keys, hashFunctionFactories).createPartitioner(),
+ numPartitions);
comparators = new IBinaryComparator[comparatorFactories.length];
for (int i = 0; i < comparatorFactories.length; ++i) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
@@ -87,8 +87,8 @@
private void write(int i, ByteBuffer head) throws HyracksDataException {
RunFileWriter writer = state.getRunWriters()[i];
if (writer == null) {
- FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
- GraceHashJoinOperatorDescriptor.class.getSimpleName());
+ FileReference file = ctx.getJobletContext()
+ .createManagedWorkspaceFile(GraceHashJoinOperatorDescriptor.class.getSimpleName());
writer = new RunFileWriter(file, ctx.getIOManager());
writer.open();
state.getRunWriters()[i] = writer;
@@ -101,8 +101,7 @@
accessor0.reset(buffer);
int tCount = accessor0.getTupleCount();
for (int i = 0; i < tCount; ++i) {
-
- int entry = hpc.partition(accessor0, i, numPartitions);
+ int entry = pu.hashPartitionKey(accessor0, i);
IFrame outbuf = outbufs[entry];
appender.reset(outbuf, false);
if (!appender.append(accessor0, i)) {
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
index f72d528..4952b64 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
@@ -60,6 +60,7 @@
import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
import org.apache.hyracks.dataflow.std.structures.ISerializableTable;
import org.apache.hyracks.dataflow.std.structures.SerializableHashTable;
+import org.apache.hyracks.dataflow.std.util.PartitionUtil;
public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor {
private static final int BUILD_AND_PARTITION_ACTIVITY_ID = 0;
@@ -188,7 +189,7 @@
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
- throws HyracksDataException {
+ throws HyracksDataException {
final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(joinAid, 0);
final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
@@ -201,15 +202,17 @@
nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
}
}
- final IPredicateEvaluator predEvaluator = (predEvaluatorFactory == null ? null : predEvaluatorFactory
- .createPredicateEvaluator());
+ final IPredicateEvaluator predEvaluator = (predEvaluatorFactory == null ? null
+ : predEvaluatorFactory.createPredicateEvaluator());
IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
- private BuildAndPartitionTaskState state = new BuildAndPartitionTaskState(ctx.getJobletContext()
- .getJobId(), new TaskId(getActivityId(), partition));
+ private BuildAndPartitionTaskState state = new BuildAndPartitionTaskState(
+ ctx.getJobletContext().getJobId(), new TaskId(getActivityId(), partition));
private final FrameTupleAccessor accessorBuild = new FrameTupleAccessor(rd1);
private final ITuplePartitionComputer hpcBuild = new FieldHashPartitionComputerFactory(keys1,
hashFunctionFactories).createPartitioner();
+ private PartitionUtil puBuildMemory;
+ private PartitionUtil puBuildAlternate;
private final FrameTupleAppender appender = new FrameTupleAppender();
private final FrameTupleAppender ftappender = new FrameTupleAppender();
private IFrame[] bufferForPartitions;
@@ -241,7 +244,7 @@
for (int i = 0; i < tCount; ++i) {
int entry;
if (state.memoryForHashtable == 0) {
- entry = hpcBuild.partition(accessorBuild, i, state.nPartitions);
+ entry = puBuildMemory.hashPartitionKey(accessorBuild, i);
boolean newBuffer = false;
IFrame bufBi = bufferForPartitions[entry];
while (true) {
@@ -255,7 +258,7 @@
}
}
} else {
- entry = hpcBuild.partition(accessorBuild, i, (int) (inputsize0 * factor / nPartitions));
+ entry = puBuildAlternate.hashPartitionKey(accessorBuild, i);
if (entry < state.memoryForHashtable) {
while (true) {
if (!ftappender.append(accessorBuild, i)) {
@@ -302,8 +305,8 @@
if (memsize > inputsize0) {
state.nPartitions = 0;
} else {
- state.nPartitions = (int) (Math.ceil((inputsize0 * factor / nPartitions - memsize)
- / (memsize - 1)));
+ state.nPartitions = (int) (Math.ceil(
+ (double) (inputsize0 * factor / nPartitions - memsize) / (double) (memsize - 1)));
}
if (state.nPartitions <= 0) {
// becomes in-memory HJ
@@ -336,6 +339,9 @@
}
ftappender.reset(inBuffer, true);
+
+ puBuildMemory = new PartitionUtil(hpcBuild, state.nPartitions);
+ puBuildAlternate = new PartitionUtil(hpcBuild, (int) (inputsize0 * factor / nPartitions));
}
@Override
@@ -352,8 +358,8 @@
private void write(int i, ByteBuffer head) throws HyracksDataException {
RunFileWriter writer = state.fWriters[i];
if (writer == null) {
- FileReference file = ctx.getJobletContext().createManagedWorkspaceFile(
- BuildAndPartitionActivityNode.class.getSimpleName());
+ FileReference file = ctx.getJobletContext()
+ .createManagedWorkspaceFile(BuildAndPartitionActivityNode.class.getSimpleName());
writer = new RunFileWriter(file, ctx.getIOManager());
writer.open();
state.fWriters[i] = writer;
@@ -378,7 +384,7 @@
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
- throws HyracksDataException {
+ throws HyracksDataException {
final RecordDescriptor rd0 = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
final RecordDescriptor rd1 = recordDescProvider.getInputRecordDescriptor(buildAid, 0);
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
@@ -391,8 +397,8 @@
nullWriters1[i] = nullWriterFactories1[i].createNullWriter();
}
}
- final IPredicateEvaluator predEvaluator = (predEvaluatorFactory == null ? null : predEvaluatorFactory
- .createPredicateEvaluator());
+ final IPredicateEvaluator predEvaluator = (predEvaluatorFactory == null ? null
+ : predEvaluatorFactory.createPredicateEvaluator());
IOperatorNodePushable op = new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
private BuildAndPartitionTaskState state;
@@ -402,7 +408,8 @@
private final ITuplePartitionComputerFactory hpcf1 = new FieldHashPartitionComputerFactory(keys1,
hashFunctionFactories);
private final ITuplePartitionComputer hpcProbe = hpcf0.createPartitioner();
-
+ private PartitionUtil puProbeMemory;
+ private PartitionUtil puProbeAlternate;
private final FrameTupleAppender appender = new FrameTupleAppender();
private final FrameTupleAppender ftap = new FrameTupleAppender();
private final IFrame inBuffer = new VSizeFrame(ctx);
@@ -413,8 +420,8 @@
@Override
public void open() throws HyracksDataException {
- state = (BuildAndPartitionTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(),
- BUILD_AND_PARTITION_ACTIVITY_ID), partition));
+ state = (BuildAndPartitionTaskState) ctx.getStateObject(
+ new TaskId(new ActivityId(getOperatorId(), BUILD_AND_PARTITION_ACTIVITY_ID), partition));
writer.open();
buildWriters = state.fWriters;
probeWriters = new RunFileWriter[state.nPartitions];
@@ -424,6 +431,9 @@
}
appender.reset(outBuffer, true);
ftap.reset(inBuffer, true);
+
+ puProbeMemory = new PartitionUtil(hpcProbe, state.nPartitions);
+ puProbeAlternate = new PartitionUtil(hpcProbe, (int) (inputsize0 * factor / nPartitions));
}
@Override
@@ -435,7 +445,7 @@
int entry;
if (state.memoryForHashtable == 0) {
- entry = hpcProbe.partition(accessorProbe, i, state.nPartitions);
+ entry = puProbeMemory.hashPartitionKey(accessorProbe, i);
boolean newBuffer = false;
IFrame outbuf = bufferForPartitions[entry];
while (true) {
@@ -449,7 +459,7 @@
}
}
} else {
- entry = hpcProbe.partition(accessorProbe, i, (int) (inputsize0 * factor / nPartitions));
+ entry = puProbeAlternate.hashPartitionKey(accessorProbe, i);
if (entry < state.memoryForHashtable) {
while (true) {
if (!ftap.append(accessorProbe, i)) {
@@ -515,8 +525,9 @@
}
table.reset();
InMemoryHashJoin joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(rd0),
- hpcRep0, new FrameTupleAccessor(rd1), hpcRep1, new FrameTuplePairComparator(keys0,
- keys1, comparators), isLeftOuter, nullWriters1, table, predEvaluator);
+ hpcRep0, new FrameTupleAccessor(rd1), hpcRep1,
+ new FrameTuplePairComparator(keys0, keys1, comparators), isLeftOuter, nullWriters1,
+ table, predEvaluator);
if (buildWriter != null) {
RunFileReader buildReader = buildWriter.createDeleteOnCloseReader();
@@ -554,8 +565,8 @@
private void write(int i, ByteBuffer head) throws HyracksDataException {
RunFileWriter writer = probeWriters[i];
if (writer == null) {
- FileReference file = ctx.createManagedWorkspaceFile(PartitionAndJoinActivityNode.class
- .getSimpleName());
+ FileReference file = ctx
+ .createManagedWorkspaceFile(PartitionAndJoinActivityNode.class.getSimpleName());
writer = new RunFileWriter(file, ctx.getIOManager());
writer.open();
probeWriters[i] = writer;
@@ -571,4 +582,4 @@
return op;
}
}
-}
\ No newline at end of file
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
new file mode 100644
index 0000000..90a95b0
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.dataflow.std.join;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IMergeJoinChecker extends Serializable {
+
+ boolean checkToSaveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex, IFrameTupleAccessor accessorRight,
+ int rightTupleIndex) throws HyracksDataException;
+
+ boolean checkToRemoveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException;
+
+ boolean checkToLoadNextRightTuple(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException;
+
+ boolean checkToSaveInResult(IFrameTupleAccessor accessorLeft, int leftTupleIndex, IFrameTupleAccessor accessorRight,
+ int rightTupleIndex) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..7d01629
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.dataflow.std.join;
+
+import java.io.Serializable;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public interface IMergeJoinCheckerFactory extends Serializable {
+
+ IMergeJoinChecker createMergeJoinChecker(IBinaryComparatorFactory[] comparatorFactories, int[] keys0, int[] keys1);
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
index 2a66fb7..63817ff 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
@@ -38,15 +38,16 @@
import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
import org.apache.hyracks.dataflow.std.structures.ISerializableTable;
import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+import org.apache.hyracks.dataflow.std.util.PartitionUtil;
public class InMemoryHashJoin {
private final IHyracksTaskContext ctx;
private final List<ByteBuffer> buffers;
private final FrameTupleAccessor accessorBuild;
- private final ITuplePartitionComputer tpcBuild;
+ private final PartitionUtil puBuild;
private final FrameTupleAccessor accessorProbe;
- private final ITuplePartitionComputer tpcProbe;
+ private final PartitionUtil puProbe;
private final FrameTupleAppender appender;
private final FrameTuplePairComparator tpComparator;
private final boolean isLeftOuter;
@@ -77,9 +78,10 @@
storedTuplePointer = new TuplePointer();
buffers = new ArrayList<ByteBuffer>();
this.accessorBuild = accessor1;
- this.tpcBuild = tpc1;
+ puBuild = new PartitionUtil(tpc1, tableSize);
+
this.accessorProbe = accessor0;
- this.tpcProbe = tpc0;
+ puProbe = new PartitionUtil(tpc0, tableSize);
appender = new FrameTupleAppender(new VSizeFrame(ctx));
tpComparator = comparator;
predEvaluator = predEval;
@@ -106,7 +108,7 @@
accessorBuild.reset(buffer);
int tCount = accessorBuild.getTupleCount();
for (int i = 0; i < tCount; ++i) {
- int entry = tpcBuild.partition(accessorBuild, i, tableSize);
+ int entry = puBuild.hashPartitionKey(accessorBuild, i);
storedTuplePointer.frameIndex = bIndex;
storedTuplePointer.tupleIndex = i;
table.insert(entry, storedTuplePointer);
@@ -119,7 +121,7 @@
for (int i = 0; i < tupleCount0; ++i) {
boolean matchFound = false;
if (tableSize != 0) {
- int entry = tpcProbe.partition(accessorProbe, i, tableSize);
+ int entry = puProbe.hashPartitionKey(accessorProbe, i);
int offset = 0;
do {
table.getTuplePointer(entry, offset++, storedTuplePointer);
@@ -139,9 +141,8 @@
} while (true);
}
if (!matchFound && isLeftOuter) {
- FrameUtils.appendConcatToWriter(writer, appender, accessorProbe, i,
- nullTupleBuild.getFieldEndOffsets(), nullTupleBuild.getByteArray(), 0,
- nullTupleBuild.getSize());
+ FrameUtils.appendConcatToWriter(writer, appender, accessorProbe, i, nullTupleBuild.getFieldEndOffsets(),
+ nullTupleBuild.getByteArray(), 0, nullTupleBuild.getSize());
}
}
}
@@ -173,11 +174,9 @@
private void appendToResult(int probeSidetIx, int buildSidetIx, IFrameWriter writer) throws HyracksDataException {
if (!reverseOutputOrder) {
- FrameUtils.appendConcatToWriter(writer, appender, accessorProbe, probeSidetIx, accessorBuild,
- buildSidetIx);
+ FrameUtils.appendConcatToWriter(writer, appender, accessorProbe, probeSidetIx, accessorBuild, buildSidetIx);
} else {
- FrameUtils.appendConcatToWriter(writer, appender, accessorBuild, buildSidetIx, accessorProbe,
- probeSidetIx);
+ FrameUtils.appendConcatToWriter(writer, appender, accessorBuild, buildSidetIx, accessorProbe, probeSidetIx);
}
}
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
new file mode 100644
index 0000000..cd2ba82
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.dataflow.std.join;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+public class MergeJoinLocks implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ private final List<Lock> lock = new ArrayList<Lock>();
+ private final List<Condition> left = new ArrayList<Condition>();
+ private final List<Condition> right = new ArrayList<Condition>();
+
+ public synchronized void setPartitions(int partitions) {
+ for (int i = lock.size(); i < partitions; ++i) {
+ lock.add(new ReentrantLock());
+ left.add(lock.get(i).newCondition());
+ right.add(lock.get(i).newCondition());
+ }
+ }
+
+ public Lock getLock(int partition) {
+ return lock.get(partition);
+ }
+
+ public Condition getLeft(int partition) {
+ return left.get(partition);
+ }
+
+ public Condition getRight(int partition) {
+ return right.get(partition);
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
new file mode 100644
index 0000000..3a8acc4
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
@@ -0,0 +1,323 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.dataflow.std.join;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.ActivityId;
+import org.apache.hyracks.api.dataflow.IActivityGraphBuilder;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.TaskId;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.dataflow.std.base.AbstractActivityNode;
+import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+
+/**
+ * The merge join is made up of two operators: left and right.
+ * The right operator loads right stream into memory for the merge process.
+ * The left operator streams the left input and the right memory store to merge and join the data.
+ *
+ * @author prestonc
+ */
+public class MergeJoinOperatorDescriptor extends AbstractOperatorDescriptor {
+ private static final int LEFT_ACTIVITY_ID = 0;
+ private static final int RIGHT_ACTIVITY_ID = 1;
+ private final IBinaryComparatorFactory[] comparatorFactories;
+ private final IMergeJoinCheckerFactory mergeJoinCheckerFactory;
+ private final int[] keys0;
+ private final int[] keys1;
+ private final int memSize;
+
+ public MergeJoinOperatorDescriptor(IOperatorDescriptorRegistry spec, int memSize, RecordDescriptor recordDescriptor,
+ int[] keys0, int[] keys1, IBinaryComparatorFactory[] comparatorFactories,
+ IMergeJoinCheckerFactory mergeJoinCheckerFactory) {
+ super(spec, 2, 1);
+ recordDescriptors[0] = recordDescriptor;
+ this.comparatorFactories = comparatorFactories;
+ this.mergeJoinCheckerFactory = mergeJoinCheckerFactory;
+ this.keys0 = keys0;
+ this.keys1 = keys1;
+ this.memSize = memSize;
+ }
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public void contributeActivities(IActivityGraphBuilder builder) {
+ MergeJoinLocks locks = new MergeJoinLocks();
+ ActivityId leftAid = new ActivityId(odId, LEFT_ACTIVITY_ID);
+ ActivityId rightAid = new ActivityId(odId, RIGHT_ACTIVITY_ID);
+ LeftJoinerActivityNode leftAN = new LeftJoinerActivityNode(leftAid, rightAid, locks);
+ RightDataActivityNode rightAN = new RightDataActivityNode(rightAid, leftAid, locks);
+
+ builder.addActivity(this, rightAN);
+ builder.addSourceEdge(1, rightAN, 0);
+
+ builder.addActivity(this, leftAN);
+ builder.addSourceEdge(0, leftAN, 0);
+ builder.addTargetEdge(0, leftAN, 0);
+ }
+
+ public static class SortMergeIntervalJoinTaskState extends AbstractStateObject {
+ private MergeStatus status;
+ private MergeJoiner joiner;
+ private boolean failed;
+
+ private SortMergeIntervalJoinTaskState(JobId jobId, TaskId taskId) {
+ super(jobId, taskId);
+ status = new MergeStatus();
+ }
+
+ @Override
+ public void toBytes(DataOutput out) throws IOException {
+
+ }
+
+ @Override
+ public void fromBytes(DataInput in) throws IOException {
+ }
+ }
+
+ private class LeftJoinerActivityNode extends AbstractActivityNode {
+ private static final long serialVersionUID = 1L;
+
+ private final ActivityId joinAid;
+ private final MergeJoinLocks locks;
+
+ public LeftJoinerActivityNode(ActivityId id, ActivityId joinAid, MergeJoinLocks locks) {
+ super(id);
+ this.joinAid = joinAid;
+ this.locks = locks;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+ throws HyracksDataException {
+ locks.setPartitions(nPartitions);
+ final RecordDescriptor inRecordDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+ IMergeJoinChecker mjc = mergeJoinCheckerFactory.createMergeJoinChecker(comparatorFactories, keys0, keys1);
+ return new LeftJoinerOperator(ctx, partition, inRecordDesc, mjc);
+ }
+
+ private class LeftJoinerOperator extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+
+ private final IHyracksTaskContext ctx;
+ private final int partition;
+ private final RecordDescriptor leftRD;
+ private final IMergeJoinChecker mjc;
+
+ public LeftJoinerOperator(IHyracksTaskContext ctx, int partition, RecordDescriptor inRecordDesc,
+ IMergeJoinChecker mjc) {
+ this.ctx = ctx;
+ this.partition = partition;
+ this.leftRD = inRecordDesc;
+ this.mjc = mjc;
+ }
+
+ private SortMergeIntervalJoinTaskState state;
+ private boolean first = true;
+
+ public void open() throws HyracksDataException {
+ locks.getLock(partition).lock();
+ try {
+ writer.open();
+ state = new SortMergeIntervalJoinTaskState(ctx.getJobletContext().getJobId(),
+ new TaskId(getActivityId(), partition));
+ state.status.openLeft();
+ state.joiner = new MergeJoiner(ctx, memSize, partition, state.status, locks, mjc, leftRD);
+ ctx.setStateObject(state);
+ locks.getRight(partition).signal();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ locks.getLock(partition).unlock();
+ }
+ }
+
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ locks.getLock(partition).lock();
+ if (first) {
+ state.status.dataLeft();
+ first = false;
+ }
+ try {
+ state.joiner.setLeftFrame(buffer);
+ state.joiner.processMergeUsingLeftTuple(writer);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ locks.getLock(partition).unlock();
+ }
+ }
+
+ public void fail() throws HyracksDataException {
+ locks.getLock(partition).lock();
+ try {
+ state.failed = true;
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ locks.getLock(partition).unlock();
+ }
+ }
+
+ public void close() throws HyracksDataException {
+ locks.getLock(partition).lock();
+ try {
+ state.status.leftHasMore = false;
+ if (state.failed) {
+ writer.fail();
+ } else {
+ state.joiner.processMergeUsingLeftTuple(writer);
+ state.joiner.closeResult(writer);
+ writer.close();
+ }
+ state.status.closeLeft();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ locks.getLock(partition).unlock();
+ }
+ }
+ }
+ }
+
+ private class RightDataActivityNode extends AbstractActivityNode {
+ private static final long serialVersionUID = 1L;
+
+ private final ActivityId joinAid;
+ private MergeJoinLocks locks;
+
+ public RightDataActivityNode(ActivityId id, ActivityId joinAid, MergeJoinLocks locks) {
+ super(id);
+ this.joinAid = joinAid;
+ this.locks = locks;
+ }
+
+ @Override
+ public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
+ IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
+ throws HyracksDataException {
+ locks.setPartitions(nPartitions);
+ RecordDescriptor inRecordDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+ return new RightDataOperator(ctx, partition, inRecordDesc);
+ }
+
+ private class RightDataOperator extends AbstractUnaryInputSinkOperatorNodePushable {
+
+ private int partition;
+ private IHyracksTaskContext ctx;
+ private final RecordDescriptor rightRD;
+
+ public RightDataOperator(IHyracksTaskContext ctx, int partition, RecordDescriptor inRecordDesc) {
+ this.ctx = ctx;
+ this.partition = partition;
+ this.rightRD = inRecordDesc;
+ }
+
+ private SortMergeIntervalJoinTaskState state;
+ private boolean first = true;
+
+ @Override
+ public void open() throws HyracksDataException {
+ locks.getLock(partition).lock();
+ try {
+ do {
+ // Wait for the state to be set in the context form Left.
+ state = (SortMergeIntervalJoinTaskState) ctx.getStateObject(new TaskId(joinAid, partition));
+ if (state == null) {
+ locks.getRight(partition).await();
+ }
+ } while (state == null);
+ state.joiner.setRightRecordDescriptor(rightRD);
+ state.status.openRight();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException("RightOperator interrupted exceptrion", e);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ locks.getLock(partition).unlock();
+ }
+ }
+
+ @Override
+ public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+ locks.getLock(partition).lock();
+ if (first) {
+ state.status.dataRight();
+ first = false;
+ }
+ try {
+ while (state.status.loadRightFrame == false && state.status.leftHasMore == true) {
+ // Wait for the state to request right frame unless left has finished.
+ locks.getRight(partition).await();
+ };
+ state.joiner.setRightFrame(buffer);
+ locks.getLeft(partition).signal();
+ } catch (InterruptedException e) {
+ throw new HyracksDataException("RightOperator interrupted exceptrion", e);
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ locks.getLock(partition).unlock();
+ }
+ }
+
+ @Override
+ public void fail() throws HyracksDataException {
+ locks.getLock(partition).lock();
+ try {
+ state.failed = true;
+ locks.getLeft(partition).signal();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ locks.getLock(partition).unlock();
+ }
+ }
+
+ @Override
+ public void close() throws HyracksDataException {
+ locks.getLock(partition).lock();
+ try {
+ state.status.closeRight();
+ locks.getLeft(partition).signal();
+ } catch (Exception e) {
+ throw new HyracksDataException(e);
+ } finally {
+ locks.getLock(partition).unlock();
+ }
+ }
+ }
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
new file mode 100644
index 0000000..50ea016
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
@@ -0,0 +1,362 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.dataflow.std.join;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hyracks.api.comm.FixedSizeFrame;
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.VSizeFrame;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.io.FileReference;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
+import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.common.io.RunFileReader;
+import org.apache.hyracks.dataflow.common.io.RunFileWriter;
+import org.apache.hyracks.dataflow.std.join.MergeStatus.BranchStatus;
+import org.apache.hyracks.dataflow.std.join.MergeStatus.RunFileStatus;
+import org.apache.hyracks.dataflow.std.sort.buffermanager.IFramePool;
+import org.apache.hyracks.dataflow.std.sort.buffermanager.IFrameTupleBufferAccessor;
+import org.apache.hyracks.dataflow.std.sort.buffermanager.ITupleBufferManager;
+import org.apache.hyracks.dataflow.std.sort.buffermanager.VariableFramePool;
+import org.apache.hyracks.dataflow.std.sort.buffermanager.VariableTupleMemoryManager;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+/**
+ * Merge Joiner takes two sorted streams of input and joins.
+ * The two sorted streams must be in a logical order and the comparator must
+ * support keeping that order so the join will work.
+ * The left stream will spill to disk when memory is full.
+ * The right stream spills to memory and pause when memory is full.
+ *
+ * @author prestonc
+ */
+public class MergeJoiner {
+
+ private static final int MEMORY_INDEX = -1;
+
+ private final FrameTupleAccessor accessorLeft;
+ private FrameTupleAccessor accessorRight;
+
+ private MergeJoinLocks locks;
+ private MergeStatus status;
+
+ private ByteBuffer leftBuffer;
+ private ByteBuffer rightBuffer;
+ private int leftBufferTupleIndex;
+ private int leftRunFileTupleIndex;
+ private int rightBufferTupleIndex;
+
+ private final TuplePointer tp;
+ private final List<TuplePointer> memoryTuples;
+ private final IFramePool framePool;
+ private ITupleBufferManager bufferManager;
+ private IFrameTupleBufferAccessor memoryAccessor;
+
+ private final IFrame runFileBuffer;
+ private final FrameTupleAppender runFileAppender;
+ private final RunFileWriter runFileWriter;
+ private RunFileReader runFileReader;
+
+ private final FrameTupleAppender resultAppender;
+
+ private final IMergeJoinChecker mjc;
+
+ private final int partition;
+
+ public MergeJoiner(IHyracksTaskContext ctx, int memorySize, int partition, MergeStatus status, MergeJoinLocks locks,
+ IMergeJoinChecker mjc, RecordDescriptor leftRd) throws HyracksDataException {
+ this.partition = partition;
+ this.status = status;
+ this.locks = locks;
+ this.mjc = mjc;
+
+ accessorLeft = new FrameTupleAccessor(leftRd);
+ leftBuffer = ctx.allocateFrame();
+ rightBuffer = ctx.allocateFrame();
+ leftBufferTupleIndex = -1;
+ rightBufferTupleIndex = -1;
+
+ // Memory (right buffer)
+ framePool = new VariableFramePool(ctx, (memorySize - 1) * ctx.getInitialFrameSize());
+ memoryTuples = new ArrayList<TuplePointer>();
+ tp = new TuplePointer();
+
+ // Run File and frame cache (left buffer)
+ FileReference file = ctx.getJobletContext()
+ .createManagedWorkspaceFile(this.getClass().getSimpleName() + this.toString());
+ runFileWriter = new RunFileWriter(file, ctx.getIOManager());
+ runFileWriter.open();
+ runFileBuffer = new FixedSizeFrame(ctx.allocateFrame(ctx.getInitialFrameSize()));
+ runFileAppender = new FrameTupleAppender(new VSizeFrame(ctx));
+ leftRunFileTupleIndex = -1;
+
+ // Result
+ resultAppender = new FrameTupleAppender(new VSizeFrame(ctx));
+ }
+
+ private boolean addToMemory(IFrameTupleAccessor accessor, int idx) throws HyracksDataException {
+ TuplePointer tuplePointer = new TuplePointer();
+ if (bufferManager.insertTuple(accessor, idx, tuplePointer)) {
+ memoryTuples.add(tuplePointer);
+ return true;
+ }
+ return false;
+ }
+
+ private void addToResult(IFrameTupleAccessor accessor1, int index1, IFrameTupleAccessor accessor2, int index2,
+ IFrameWriter writer) throws HyracksDataException {
+ FrameUtils.appendConcatToWriter(writer, resultAppender, accessor1, index1, accessor2, index2);
+ }
+
+ public void closeResult(IFrameWriter writer) throws HyracksDataException {
+ resultAppender.flush(writer, true);
+ }
+
+ private void addToRunFile(IFrameTupleAccessor accessor, int idx) throws HyracksDataException {
+ if (!runFileAppender.append(accessor, idx)) {
+ runFileAppender.flush(runFileWriter, true);
+ runFileAppender.append(accessor, idx);
+ }
+ }
+
+ private void openRunFile() throws HyracksDataException {
+ status.runFileStatus = RunFileStatus.READING;
+
+ // Create reader
+ runFileReader = runFileWriter.createReader();
+ runFileReader.open();
+ leftRunFileTupleIndex = 0;
+
+ // Load first frame
+ runFileReader.nextFrame(runFileBuffer);
+ accessorLeft.reset(runFileBuffer.getBuffer());
+ }
+
+ private void closeRunFile() throws HyracksDataException {
+ status.runFileStatus = RunFileStatus.NOT_USED;
+ runFileReader.close();
+ accessorLeft.reset(leftBuffer);
+ }
+
+ private void flushMemory() throws HyracksDataException {
+ bufferManager.reset();
+ memoryTuples.clear();
+ }
+
+ private int getRightTupleIndex() throws HyracksDataException {
+ return rightBufferTupleIndex;
+ }
+
+ private void incrementRightTuple() throws HyracksDataException {
+ ++rightBufferTupleIndex;
+ }
+
+ /**
+ * Ensures a frame exists for the right branch, either from memory or the run file.
+ *
+ * @throws HyracksDataException
+ */
+ private boolean loadRightTuple() throws HyracksDataException {
+ boolean loaded = true;
+ if ((rightBufferTupleIndex == -1 || rightBufferTupleIndex >= accessorRight.getTupleCount())
+ && status.rightHasMore == true) {
+ status.loadRightFrame = true;
+ locks.getRight(partition).signal();
+ try {
+ while (status.loadRightFrame && status.getRightStatus().isEqualOrBefore(BranchStatus.DATA_PROCESSING)) {
+ locks.getLeft(partition).await();
+ }
+ } catch (InterruptedException e) {
+ throw new HyracksDataException(
+ "SortMergeIntervalJoin interrupted exception while attempting to load right tuple", e);
+ }
+ loaded = (rightBufferTupleIndex == 0);
+ if (!loaded) {
+ status.rightHasMore = false;
+ }
+ }
+ return loaded;
+ }
+
+ private int getLeftTupleIndex() throws HyracksDataException {
+ if (status.runFileStatus == RunFileStatus.READING) {
+ return leftRunFileTupleIndex;
+ } else {
+ return leftBufferTupleIndex;
+ }
+ }
+
+ private void incrementLeftTuple() throws HyracksDataException {
+ if (status.runFileStatus == RunFileStatus.READING) {
+ ++leftRunFileTupleIndex;
+ } else {
+ ++leftBufferTupleIndex;
+ }
+ }
+
+ /**
+ * Ensures a frame exists for the right branch, either from memory or the run file.
+ *
+ * @throws HyracksDataException
+ */
+ private boolean loadLeftTuple() throws HyracksDataException {
+ boolean loaded = true;
+ if (status.runFileStatus == RunFileStatus.READING) {
+ if (leftRunFileTupleIndex >= accessorLeft.getTupleCount()) {
+ if (runFileReader.nextFrame(runFileBuffer)) {
+ accessorLeft.reset(runFileBuffer.getBuffer());
+ leftRunFileTupleIndex = 0;
+ } else {
+ closeRunFile();
+ return loadLeftTuple();
+ }
+ }
+ } else {
+ if (leftBufferTupleIndex == -1 || leftBufferTupleIndex >= accessorLeft.getTupleCount()) {
+ loaded = false;
+ }
+ }
+ return loaded;
+ }
+
+ /**
+ * Ensures a frame exists for the right branch, either from memory or the run file.
+ *
+ * @throws HyracksDataException
+ */
+ private boolean needNewLeftTuple() throws HyracksDataException {
+ if (leftBufferTupleIndex == -1 || leftBufferTupleIndex >= accessorLeft.getTupleCount()) {
+ return true;
+ }
+ return false;
+ }
+
+ // memory management
+ private boolean memoryHasTuples() {
+ return bufferManager.getNumTuples() > 0;
+ }
+
+ /**
+ * Left
+ *
+ * @throws HyracksDataException
+ */
+ public void processMergeUsingLeftTuple(IFrameWriter writer) throws HyracksDataException {
+ // *********************
+ // Left side from tuple
+ // *********************
+ while (!needNewLeftTuple() && loadLeftTuple()) {
+ // Write left tuple to run file
+ if (status.runFileStatus == RunFileStatus.WRITING) {
+ addToRunFile(accessorLeft, getLeftTupleIndex());
+ }
+
+ // *********************
+ // Right side from memory
+ // *********************
+ if (rightBufferTupleIndex != -1) {
+ Iterator<TuplePointer> memoryIterator = memoryTuples.iterator();
+ while (memoryIterator.hasNext()) {
+ tp.reset(memoryIterator.next());
+ memoryAccessor.reset(tp);
+ if (mjc.checkToSaveInResult(accessorLeft, getLeftTupleIndex(), memoryAccessor, MEMORY_INDEX)) {
+ // add to result
+ addToResult(accessorLeft, getLeftTupleIndex(), memoryAccessor, MEMORY_INDEX, writer);
+ }
+ if (mjc.checkToRemoveInMemory(accessorLeft, getLeftTupleIndex(), memoryAccessor, MEMORY_INDEX)) {
+ // remove from memory
+ bufferManager.deleteTuple(tp);
+ memoryIterator.remove();
+ }
+ }
+
+ // Memory is empty and we can start processing the run file.
+ if (!memoryHasTuples() && status.runFileStatus == RunFileStatus.WRITING) {
+ openRunFile();
+ flushMemory();
+ }
+ }
+
+ // *********************
+ // Right side from stream
+ // *********************
+ if (status.runFileStatus == RunFileStatus.NOT_USED && loadRightTuple() && status.rightHasMore) {
+ while (mjc.checkToLoadNextRightTuple(accessorLeft, getLeftTupleIndex(), accessorRight,
+ getRightTupleIndex())) {
+ if (mjc.checkToSaveInResult(accessorLeft, getLeftTupleIndex(), accessorRight,
+ getRightTupleIndex())) {
+ // add to result
+ addToResult(accessorLeft, getLeftTupleIndex(), accessorRight, getRightTupleIndex(), writer);
+ }
+ // append to memory
+ if (mjc.checkToSaveInMemory(accessorLeft, getLeftTupleIndex(), accessorRight,
+ getRightTupleIndex())) {
+ if (!addToMemory(accessorRight, getRightTupleIndex())) {
+ // go to log saving state
+ status.runFileStatus = RunFileStatus.WRITING;
+ // write right tuple to run file
+ addToRunFile(accessorLeft, getLeftTupleIndex());
+ break;
+ }
+ }
+ incrementRightTuple();
+ if (!loadRightTuple()) {
+ break;
+ }
+ }
+ }
+ incrementLeftTuple();
+ }
+ }
+
+ public void setLeftFrame(ByteBuffer buffer) {
+ if (leftBuffer.capacity() < buffer.capacity()) {
+ leftBuffer.limit(buffer.capacity());
+ }
+ leftBuffer.put(buffer.array(), 0, buffer.capacity());
+ accessorLeft.reset(leftBuffer);
+ leftBufferTupleIndex = 0;
+ }
+
+ public void setRightFrame(ByteBuffer buffer) {
+ if (rightBuffer.capacity() < buffer.capacity()) {
+ rightBuffer.limit(buffer.capacity());
+ }
+ rightBuffer.put(buffer.array(), 0, buffer.capacity());
+ accessorRight.reset(rightBuffer);
+ rightBufferTupleIndex = 0;
+ status.loadRightFrame = false;
+ }
+
+ public void setRightRecordDescriptor(RecordDescriptor rightRd) {
+ accessorRight = new FrameTupleAccessor(rightRd);
+ bufferManager = new VariableTupleMemoryManager(framePool, rightRd);
+ memoryAccessor = bufferManager.getFrameTupleAccessor();
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
new file mode 100644
index 0000000..76c5570
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.dataflow.std.join;
+
+import java.io.Serializable;
+
+public class MergeStatus implements Serializable {
+ private static final long serialVersionUID = 1L;
+
+ public enum BranchStatus {
+ UNKNOWN,
+ OPENED,
+ DATA_PROCESSING,
+ CLOSED;
+
+ public boolean isEqualOrBefore(BranchStatus bs) {
+ return this.ordinal() <= bs.ordinal();
+ }
+ }
+
+ public enum RunFileStatus {
+ NOT_USED,
+ WRITING,
+ READING,
+ }
+
+ public boolean reloadingLeftFrame = false;
+ public boolean loadRightFrame = false;
+
+ public boolean leftHasMore = true;
+ public boolean rightHasMore = true;
+
+ private BranchStatus leftStatus = BranchStatus.UNKNOWN;
+ private BranchStatus rightStatus = BranchStatus.UNKNOWN;
+
+ public RunFileStatus runFileStatus = RunFileStatus.NOT_USED;
+
+ public MergeStatus() {
+ }
+
+ public BranchStatus getLeftStatus() {
+ return leftStatus;
+ }
+
+ public BranchStatus getRightStatus() {
+ return rightStatus;
+ }
+
+ public void openLeft() {
+ leftStatus = BranchStatus.OPENED;
+ }
+
+ public void openRight() {
+ rightStatus = BranchStatus.OPENED;
+ }
+
+ public void dataLeft() {
+ leftStatus = BranchStatus.DATA_PROCESSING;
+ }
+
+ public void dataRight() {
+ rightStatus = BranchStatus.DATA_PROCESSING;
+ }
+
+ public void closeLeft() {
+ leftStatus = BranchStatus.CLOSED;
+ }
+
+ public void closeRight() {
+ rightStatus = BranchStatus.CLOSED;
+ }
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
new file mode 100644
index 0000000..798713a
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.dataflow.std.join;
+
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
+
+public class NaturalMergeJoinChecker implements IMergeJoinChecker {
+ private static final long serialVersionUID = 1L;
+ FrameTuplePairComparator comparator;
+
+ public NaturalMergeJoinChecker(FrameTuplePairComparator comparator) {
+ this.comparator = comparator;
+ }
+
+ public boolean checkToSaveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException {
+ int c = comparator.compare(accessorLeft, leftTupleIndex, accessorRight, rightTupleIndex);
+ return (c == 0);
+ }
+
+ public boolean checkToRemoveInMemory(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException {
+ int c = comparator.compare(accessorLeft, leftTupleIndex, accessorRight, rightTupleIndex);
+ return (c < 0);
+ }
+
+ public boolean checkToLoadNextRightTuple(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException {
+ int c = comparator.compare(accessorLeft, leftTupleIndex, accessorRight, rightTupleIndex);
+ return (c <= 0);
+ }
+
+ @Override
+ public boolean checkToSaveInResult(IFrameTupleAccessor accessorLeft, int leftTupleIndex,
+ IFrameTupleAccessor accessorRight, int rightTupleIndex) throws HyracksDataException {
+ int c = comparator.compare(accessorLeft, leftTupleIndex, accessorRight, rightTupleIndex);
+ return (c == 0);
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
new file mode 100644
index 0000000..a1d303a
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.dataflow.std.join;
+
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
+
+public class NaturalMergeJoinCheckerFactory implements IMergeJoinCheckerFactory {
+
+ @Override
+ public IMergeJoinChecker createMergeJoinChecker(IBinaryComparatorFactory[] comparatorFactories, int[] keys0,
+ int[] keys1) {
+ final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
+ for (int i = 0; i < comparatorFactories.length; ++i) {
+ comparators[i] = comparatorFactories[i].createBinaryComparator();
+ }
+ FrameTuplePairComparator ftp = new FrameTuplePairComparator(keys0, keys1, comparators);
+ return new NaturalMergeJoinChecker(ftp);
+ }
+
+}
\ No newline at end of file
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
index 58e1b29..7dbeef9 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
@@ -43,6 +43,7 @@
import org.apache.hyracks.dataflow.common.io.RunFileWriter;
import org.apache.hyracks.dataflow.std.structures.ISerializableTable;
import org.apache.hyracks.dataflow.std.structures.SerializableHashTable;
+import org.apache.hyracks.dataflow.std.util.PartitionUtil;
/**
* @author pouria
@@ -69,6 +70,8 @@
private ITuplePartitionComputer buildHpc;
private ITuplePartitionComputer probeHpc;
+ private final PartitionUtil puBuild;
+ private final PartitionUtil puProbe;
private final RecordDescriptor buildRd;
private final RecordDescriptor probeRd;
@@ -120,6 +123,8 @@
this.probeRd = probeRd;
this.buildHpc = buildHpc;
this.probeHpc = probeHpc;
+ this.puBuild = new PartitionUtil(buildHpc, numOfPartitions);
+ this.puProbe = new PartitionUtil(probeHpc, numOfPartitions);
this.buildKeys = keys1;
this.probeKeys = keys0;
this.comparators = comparators;
@@ -150,6 +155,8 @@
this.probeRd = probeRd;
this.buildHpc = buildHpc;
this.probeHpc = probeHpc;
+ this.puBuild = new PartitionUtil(buildHpc, numOfPartitions);
+ this.puProbe = new PartitionUtil(probeHpc, numOfPartitions);
this.buildKeys = keys1;
this.probeKeys = keys0;
this.comparators = comparators;
@@ -211,7 +218,7 @@
}
for (int i = 0; i < tupleCount; ++i) {
- int pid = buildHpc.partition(accessorBuild, i, numOfPartitions);
+ int pid = puBuild.hashPartitionKey(accessorBuild, i);
processTuple(i, pid);
buildPSizeInTups[pid]++;
}
@@ -521,7 +528,7 @@
return;
}
for (int i = 0; i < tupleCount; ++i) {
- int pid = probeHpc.partition(accessorProbe, i, numOfPartitions);
+ int pid = puProbe.hashPartitionKey(accessorProbe, i);
if (buildPSizeInTups[pid] > 0 || isLeftOuter) { //Tuple has potential match from previous phase
if (pStatus.get(pid)) { //pid is Spilled
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
index 0278f92..e120faa 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
@@ -43,7 +43,6 @@
import org.apache.hyracks.api.dataflow.value.ITuplePairComparator;
import org.apache.hyracks.api.dataflow.value.ITuplePairComparatorFactory;
import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFamily;
import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
import org.apache.hyracks.api.exceptions.HyracksDataException;
import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
@@ -52,7 +51,6 @@
import org.apache.hyracks.dataflow.common.comm.io.FrameTuplePairComparator;
import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
import org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFamily;
-import org.apache.hyracks.dataflow.common.data.partition.RepartitionComputerFamily;
import org.apache.hyracks.dataflow.common.io.RunFileReader;
import org.apache.hyracks.dataflow.std.base.AbstractActivityNode;
import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
@@ -160,7 +158,7 @@
IBinaryComparatorFactory[] comparatorFactories, RecordDescriptor recordDescriptor,
ITuplePairComparatorFactory tupPaircomparatorFactory0,
ITuplePairComparatorFactory tupPaircomparatorFactory1, IPredicateEvaluatorFactory predEvaluatorFactory)
- throws HyracksDataException {
+ throws HyracksDataException {
super(spec, 2, 1);
this.memsize = memsize;
@@ -207,8 +205,8 @@
if (memorySize > buildSize) {
return 1; //We will switch to in-Mem HJ eventually
}
- numberOfPartitions = (int) (Math.ceil((double) (buildSize * factor / nPartitions - memorySize)
- / (double) (memorySize - 1)));
+ numberOfPartitions = (int) (Math
+ .ceil((double) (buildSize * factor / nPartitions - memorySize) / (double) (memorySize - 1)));
if (numberOfPartitions <= 0) {
numberOfPartitions = 1; //becomes in-memory hash join
}
@@ -273,12 +271,12 @@
comparators[i] = comparatorFactories[i].createBinaryComparator();
}
- final IPredicateEvaluator predEvaluator = (predEvaluatorFactory == null ? null : predEvaluatorFactory
- .createPredicateEvaluator());
+ final IPredicateEvaluator predEvaluator = (predEvaluatorFactory == null ? null
+ : predEvaluatorFactory.createPredicateEvaluator());
IOperatorNodePushable op = new AbstractUnaryInputSinkOperatorNodePushable() {
- private BuildAndPartitionTaskState state = new BuildAndPartitionTaskState(ctx.getJobletContext()
- .getJobId(), new TaskId(getActivityId(), partition));
+ private BuildAndPartitionTaskState state = new BuildAndPartitionTaskState(
+ ctx.getJobletContext().getJobId(), new TaskId(getActivityId(), partition));
ITuplePartitionComputer probeHpc = new FieldHashPartitionComputerFamily(probeKeys,
hashFunctionGeneratorFactories).createPartitioner(0);
@@ -351,15 +349,15 @@
@Override
public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions)
- throws HyracksDataException {
+ throws HyracksDataException {
final RecordDescriptor probeRd = recordDescProvider.getInputRecordDescriptor(buildAid, 0);
final RecordDescriptor buildRd = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
final ITuplePairComparator nljComparator0 = tuplePairComparatorFactory0.createTuplePairComparator(ctx);
final ITuplePairComparator nljComparator1 = tuplePairComparatorFactory1.createTuplePairComparator(ctx);
- final IPredicateEvaluator predEvaluator = (predEvaluatorFactory == null ? null : predEvaluatorFactory
- .createPredicateEvaluator());
+ final IPredicateEvaluator predEvaluator = (predEvaluatorFactory == null ? null
+ : predEvaluatorFactory.createPredicateEvaluator());
for (int i = 0; i < comparatorFactories.length; i++) {
comparators[i] = comparatorFactories[i].createBinaryComparator();
@@ -378,8 +376,8 @@
@Override
public void open() throws HyracksDataException {
- state = (BuildAndPartitionTaskState) ctx.getStateObject(new TaskId(new ActivityId(getOperatorId(),
- BUILD_AND_PARTITION_ACTIVITY_ID), partition));
+ state = (BuildAndPartitionTaskState) ctx.getStateObject(
+ new TaskId(new ActivityId(getOperatorId(), BUILD_AND_PARTITION_ACTIVITY_ID), partition));
writer.open();
state.hybridHJ.initProbe();
@@ -409,8 +407,7 @@
RunFileReader bReader = state.hybridHJ.getBuildRFReader(pid);
RunFileReader pReader = state.hybridHJ.getProbeRFReader(pid);
- if (bReader == null || pReader
- == null) { //either of sides (or both) does not have any tuple, thus no need for joining (no potential match)
+ if (bReader == null || pReader == null) { //either of sides (or both) does not have any tuple, thus no need for joining (no potential match)
continue;
}
int bSize = state.hybridHJ.getBuildPartitionSizeInTup(pid);
@@ -424,20 +421,16 @@
private void joinPartitionPair(OptimizedHybridHashJoin ohhj, RunFileReader buildSideReader,
RunFileReader probeSideReader, int pid, int beforeMax, int level, boolean wasReversed)
- throws HyracksDataException {
+ throws HyracksDataException {
ITuplePartitionComputer probeHpc = new FieldHashPartitionComputerFamily(probeKeys,
hashFunctionGeneratorFactories).createPartitioner(level);
ITuplePartitionComputer buildHpc = new FieldHashPartitionComputerFamily(buildKeys,
hashFunctionGeneratorFactories).createPartitioner(level);
- long buildPartSize = wasReversed ?
- (ohhj.getProbePartitionSize(pid) / ctx.getInitialFrameSize()) :
- (ohhj
- .getBuildPartitionSize(pid) / ctx.getInitialFrameSize());
- long probePartSize = wasReversed ?
- (ohhj.getBuildPartitionSize(pid) / ctx.getInitialFrameSize()) :
- (ohhj
- .getProbePartitionSize(pid) / ctx.getInitialFrameSize());
+ long buildPartSize = wasReversed ? (ohhj.getProbePartitionSize(pid) / ctx.getInitialFrameSize())
+ : (ohhj.getBuildPartitionSize(pid) / ctx.getInitialFrameSize());
+ long probePartSize = wasReversed ? (ohhj.getBuildPartitionSize(pid) / ctx.getInitialFrameSize())
+ : (ohhj.getProbePartitionSize(pid) / ctx.getInitialFrameSize());
LOGGER.fine("\n>>>Joining Partition Pairs (thread_id " + Thread.currentThread().getId() + ") (pid "
+ pid + ") - (level " + level + ") - wasReversed " + wasReversed + " - BuildSize:\t"
@@ -448,12 +441,11 @@
if (!skipInMemoryHJ && (buildPartSize < state.memForJoin)
|| (probePartSize < state.memForJoin && !isLeftOuter)) {
int tabSize = -1;
- if (!forceRR && (isLeftOuter || (buildPartSize
- < probePartSize))) { //Case 1.1 - InMemHJ (wout Role-Reversal)
+ if (!forceRR && (isLeftOuter || (buildPartSize < probePartSize))) { //Case 1.1 - InMemHJ (wout Role-Reversal)
LOGGER.fine("\t>>>Case 1.1 (IsLeftOuter || buildSize<probe) AND ApplyInMemHJ - [Level "
+ level + "]");
- tabSize = wasReversed ? ohhj.getProbePartitionSizeInTup(pid) : ohhj
- .getBuildPartitionSizeInTup(pid);
+ tabSize = wasReversed ? ohhj.getProbePartitionSizeInTup(pid)
+ : ohhj.getBuildPartitionSizeInTup(pid);
if (tabSize == 0) {
throw new HyracksDataException(
"Trying to join an empty partition. Invalid table size for inMemoryHashJoin.");
@@ -465,8 +457,8 @@
LOGGER.fine(
"\t>>>Case 1.2. (NoIsLeftOuter || probe<build) AND ApplyInMemHJ WITH RoleReversal - [Level "
+ level + "]");
- tabSize = wasReversed ? ohhj.getBuildPartitionSizeInTup(pid) : ohhj
- .getProbePartitionSizeInTup(pid);
+ tabSize = wasReversed ? ohhj.getBuildPartitionSizeInTup(pid)
+ : ohhj.getProbePartitionSizeInTup(pid);
if (tabSize == 0) {
throw new HyracksDataException(
"Trying to join an empty partition. Invalid table size for inMemoryHashJoin.");
@@ -480,8 +472,7 @@
else {
LOGGER.fine("\t>>>Case 2. ApplyRecursiveHHJ - [Level " + level + "]");
OptimizedHybridHashJoin rHHj;
- if (!forceRR && (isLeftOuter
- || buildPartSize < probePartSize)) { //Case 2.1 - Recursive HHJ (wout Role-Reversal)
+ if (!forceRR && (isLeftOuter || buildPartSize < probePartSize)) { //Case 2.1 - Recursive HHJ (wout Role-Reversal)
LOGGER.fine("\t\t>>>Case 2.1 - RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
+ level + "]");
int n = getNumberOfPartitions(state.memForJoin, (int) buildPartSize, fudgeFactor,
@@ -513,13 +504,12 @@
: maxAfterProbeSize;
BitSet rPStatus = rHHj.getPartitionStatus();
- if (!forceNLJ && (afterMax < (NLJ_SWITCH_THRESHOLD
- * beforeMax))) { //Case 2.1.1 - Keep applying HHJ
+ if (!forceNLJ && (afterMax < (NLJ_SWITCH_THRESHOLD * beforeMax))) { //Case 2.1.1 - Keep applying HHJ
LOGGER.fine(
"\t\t>>>Case 2.1.1 - KEEP APPLYING RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
+ level + "]");
- for (int rPid = rPStatus.nextSetBit(0);
- rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+ for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus
+ .nextSetBit(rPid + 1)) {
RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
@@ -527,16 +517,15 @@
continue;
}
- joinPartitionPair(rHHj, rbrfw, rprfw, rPid, afterMax, (level + 1),
- false); //checked-confirmed
+ joinPartitionPair(rHHj, rbrfw, rprfw, rPid, afterMax, (level + 1), false); //checked-confirmed
}
} else { //Case 2.1.2 - Switch to NLJ
LOGGER.fine(
"\t\t>>>Case 2.1.2 - SWITCHED to NLJ RecursiveHHJ WITH (isLeftOuter || build<probe) - [Level "
+ level + "]");
- for (int rPid = rPStatus.nextSetBit(0);
- rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+ for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus
+ .nextSetBit(rPid + 1)) {
RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
@@ -585,12 +574,11 @@
: maxAfterProbeSize;
BitSet rPStatus = rHHj.getPartitionStatus();
- if (!forceNLJ && (afterMax < (NLJ_SWITCH_THRESHOLD
- * beforeMax))) { //Case 2.2.1 - Keep applying HHJ
+ if (!forceNLJ && (afterMax < (NLJ_SWITCH_THRESHOLD * beforeMax))) { //Case 2.2.1 - Keep applying HHJ
LOGGER.fine("\t\t>>>Case 2.2.1 - KEEP APPLYING RecursiveHHJ WITH RoleReversal - [Level "
+ level + "]");
- for (int rPid = rPStatus.nextSetBit(0);
- rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+ for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus
+ .nextSetBit(rPid + 1)) {
RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
@@ -598,15 +586,14 @@
continue;
}
- joinPartitionPair(rHHj, rprfw, rbrfw, rPid, afterMax, (level + 1),
- true); //checked-confirmed
+ joinPartitionPair(rHHj, rprfw, rbrfw, rPid, afterMax, (level + 1), true); //checked-confirmed
}
} else { //Case 2.2.2 - Switch to NLJ
LOGGER.fine(
"\t\t>>>Case 2.2.2 - SWITCHED to NLJ RecursiveHHJ WITH RoleReversal - [Level "
+ level + "]");
- for (int rPid = rPStatus.nextSetBit(0);
- rPid >= 0; rPid = rPStatus.nextSetBit(rPid + 1)) {
+ for (int rPid = rPStatus.nextSetBit(0); rPid >= 0; rPid = rPStatus
+ .nextSetBit(rPid + 1)) {
RunFileReader rbrfw = rHHj.getBuildRFReader(rPid);
RunFileReader rprfw = rHHj.getProbeRFReader(rPid);
@@ -644,8 +631,7 @@
bReader.open();
rPartbuff.reset();
while (bReader.nextFrame(rPartbuff)) {
- ByteBuffer copyBuffer = ctx
- .allocateFrame(rPartbuff.getFrameSize()); //We need to allocate a copyBuffer, because this buffer gets added to the buffers list in the InMemoryHashJoin
+ ByteBuffer copyBuffer = ctx.allocateFrame(rPartbuff.getFrameSize()); //We need to allocate a copyBuffer, because this buffer gets added to the buffers list in the InMemoryHashJoin
FrameUtils.copyAndFlip(rPartbuff.getBuffer(), copyBuffer);
joiner.build(copyBuffer);
rPartbuff.reset();
@@ -665,10 +651,9 @@
private void applyNestedLoopJoin(RecordDescriptor outerRd, RecordDescriptor innerRd, int memorySize,
RunFileReader outerReader, RunFileReader innerReader, ITuplePairComparator nljComparator,
boolean reverse) throws HyracksDataException {
- NestedLoopJoin nlj = new NestedLoopJoin(ctx,
- new FrameTupleAccessor(outerRd),
- new FrameTupleAccessor(innerRd), nljComparator, memorySize,
- predEvaluator, isLeftOuter, nullWriters1);
+ NestedLoopJoin nlj = new NestedLoopJoin(ctx, new FrameTupleAccessor(outerRd),
+ new FrameTupleAccessor(innerRd), nljComparator, memorySize, predEvaluator, isLeftOuter,
+ nullWriters1);
nlj.setIsReversed(reverse);
IFrame cacheBuff = new VSizeFrame(ctx);
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
index d44826e..e349d65 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
@@ -195,8 +195,8 @@
return budget;
}
- private void prepareFrames(int extraFreeMem, List<GroupVSizeFrame> inFrames, List<RunAndMaxFrameSizePair> patialRuns)
- throws HyracksDataException {
+ private void prepareFrames(int extraFreeMem, List<GroupVSizeFrame> inFrames,
+ List<RunAndMaxFrameSizePair> patialRuns) throws HyracksDataException {
if (extraFreeMem > 0 && patialRuns.size() > 1) {
int extraFrames = extraFreeMem / ctx.getInitialFrameSize();
int avg = (extraFrames / patialRuns.size()) * ctx.getInitialFrameSize();
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
index 5c31f89..be6d097 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
@@ -124,8 +124,8 @@
INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories)
throws HyracksDataException {
this.bufferManager = bufferManager;
- this.bufferAccessor1 = bufferManager.getTupleAccessor();
- this.bufferAccessor2 = bufferManager.getTupleAccessor();
+ this.bufferAccessor1 = bufferManager.getTupleBufferAccessor();
+ this.bufferAccessor2 = bufferManager.getTupleBufferAccessor();
this.topK = topK;
this.outputFrame = new VSizeFrame(ctx);
this.outputAppender = new FrameTupleAppender();
@@ -253,9 +253,8 @@
for (int i = 0; i < numEntries; i++) {
HeapEntry minEntry = (HeapEntry) entries[i];
bufferAccessor1.reset(minEntry.tuplePointer);
- int flushed = FrameUtils
- .appendToWriter(writer, outputAppender, bufferAccessor1.getTupleBuffer().array(),
- bufferAccessor1.getTupleStartOffset(), bufferAccessor1.getTupleLength());
+ int flushed = FrameUtils.appendToWriter(writer, outputAppender, bufferAccessor1.getTupleBuffer().array(),
+ bufferAccessor1.getTupleStartOffset(), bufferAccessor1.getTupleLength());
if (flushed > 0) {
maxFrameSize = Math.max(maxFrameSize, flushed);
io++;
@@ -264,8 +263,7 @@
maxFrameSize = Math.max(maxFrameSize, outputFrame.getFrameSize());
outputAppender.flush(writer, true);
if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(
- "Flushed records:" + numEntries + "; Flushed through " + (io + 1) + " frames");
+ LOGGER.info("Flushed records:" + numEntries + "; Flushed through " + (io + 1) + " frames");
}
return maxFrameSize;
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameTupleBufferAccessor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameTupleBufferAccessor.java
new file mode 100644
index 0000000..8011e2d
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameTupleBufferAccessor.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.dataflow.std.sort.buffermanager;
+
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+
+public interface IFrameTupleBufferAccessor extends IFrameTupleAccessor {
+
+ void reset(TuplePointer tuplePointer);
+
+}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
index ae502a0..52ae265 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
@@ -42,5 +42,7 @@
void close();
- ITupleBufferAccessor getTupleAccessor();
+ ITupleBufferAccessor getTupleBufferAccessor();
+
+ IFrameTupleBufferAccessor getFrameTupleAccessor();
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
index 20642bf..a8ba7ed 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
@@ -70,8 +70,7 @@
}
@Override
- public boolean insertTuple(IFrameTupleAccessor fta, int idx, TuplePointer tuplePointer)
- throws HyracksDataException {
+ public boolean insertTuple(IFrameTupleAccessor fta, int idx, TuplePointer tuplePointer) throws HyracksDataException {
int requiredFreeSpace = calculatePhysicalSpace(fta, idx);
int frameId = findAvailableFrame(requiredFreeSpace);
if (frameId < 0) {
@@ -165,7 +164,7 @@
}
@Override
- public ITupleBufferAccessor getTupleAccessor() {
+ public ITupleBufferAccessor getTupleBufferAccessor() {
return new ITupleBufferAccessor() {
private IAppendDeletableFrameTupleAccessor bufferAccessor = new DeletableFrameTupleAppender(
recordDescriptor);
@@ -204,4 +203,77 @@
};
}
+ @Override
+ public IFrameTupleBufferAccessor getFrameTupleAccessor() {
+ return new IFrameTupleBufferAccessor() {
+ private IAppendDeletableFrameTupleAccessor bufferAccessor = new DeletableFrameTupleAppender(
+ recordDescriptor);
+ private int tid;
+
+ @Override
+ public int getAbsoluteFieldStartOffset(int tupleIndex, int fieldId) {
+ return bufferAccessor.getAbsoluteFieldStartOffset(tid, fieldId);
+ }
+
+ @Override
+ public ByteBuffer getBuffer() {
+ return bufferAccessor.getBuffer();
+ }
+
+ @Override
+ public int getFieldCount() {
+ return bufferAccessor.getFieldCount();
+ }
+
+ @Override
+ public int getFieldEndOffset(int tupleIndex, int fieldId) {
+ return bufferAccessor.getFieldEndOffset(tid, fieldId);
+ }
+
+ @Override
+ public int getFieldLength(int tupleIndex, int fieldId) {
+ return bufferAccessor.getFieldLength(tid, fieldId);
+ }
+
+ @Override
+ public int getFieldSlotsLength() {
+ return bufferAccessor.getFieldSlotsLength();
+ }
+
+ @Override
+ public int getFieldStartOffset(int tupleIndex, int fieldId) {
+ return bufferAccessor.getFieldStartOffset(tid, fieldId);
+ }
+
+ @Override
+ public int getTupleCount() {
+ return 1;
+ }
+
+ @Override
+ public int getTupleEndOffset(int tupleIndex) {
+ return bufferAccessor.getTupleEndOffset(tid);
+ }
+
+ @Override
+ public int getTupleLength(int tupleIndex) {
+ return bufferAccessor.getTupleLength(tid);
+ }
+
+ @Override
+ public int getTupleStartOffset(int tupleIndex) {
+ return bufferAccessor.getTupleStartOffset(tid);
+ }
+
+ @Override
+ public void reset(ByteBuffer buffer) {
+ }
+
+ public void reset(TuplePointer tuplePointer) {
+ bufferAccessor.reset(frames.get(tuplePointer.frameIndex));
+ tid = tuplePointer.tupleIndex;
+ }
+ };
+ }
+
}
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
index 3457fe8..a0e0201 100644
--- a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
@@ -106,7 +106,7 @@
@Override
public void fail() throws HyracksDataException {
synchronized (UnionOperator.this) {
- if (failed) {
+ if (!failed) {
writer.fail();
}
failed = true;
diff --git a/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/util/PartitionUtil.java b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/util/PartitionUtil.java
new file mode 100644
index 0000000..13cd212
--- /dev/null
+++ b/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/util/PartitionUtil.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 at
+ *
+ * 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 org.apache.hyracks.dataflow.std.util;
+
+import java.util.ArrayList;
+
+import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+
+public class PartitionUtil {
+
+ private final ArrayList<Integer> map;
+ private ITuplePartitionComputer tpc;
+ private int tableSize;
+
+ public PartitionUtil(ITuplePartitionComputer tpc, int tableSize) {
+ this.tpc = tpc;
+ this.tableSize = tableSize;
+ map = new ArrayList<Integer>();
+ }
+
+ /**
+ * Ensures the partition function only returns one result.
+ */
+ public int hashPartitionKey(FrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+ map.clear();
+ tpc.partition(accessor, tIndex, tableSize, map);
+ if (map.size() != 1) {
+ throw new HyracksDataException("Partition function did not return exactly one result.");
+ }
+ return map.get(0);
+ }
+}
diff --git a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
index 61b29fd..dbb2621 100644
--- a/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
+++ b/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
@@ -45,8 +45,8 @@
import org.apache.hyracks.dataflow.std.structures.TuplePointer;
public class VariableTupleMemoryManagerTest {
- ISerializerDeserializer[] fieldsSerDer = new ISerializerDeserializer[] {
- IntegerSerializerDeserializer.INSTANCE, UTF8StringSerializerDeserializer.INSTANCE };
+ ISerializerDeserializer[] fieldsSerDer = new ISerializerDeserializer[] { IntegerSerializerDeserializer.INSTANCE,
+ UTF8StringSerializerDeserializer.INSTANCE };
RecordDescriptor recordDescriptor = new RecordDescriptor(fieldsSerDer);
ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(recordDescriptor.getFieldCount());
VariableTupleMemoryManager tupleMemoryManager;
@@ -123,7 +123,7 @@
}
private void assertEachTupleInFTAIsInBuffer(Map<Integer, Integer> map, Map<TuplePointer, Integer> mapInserted) {
- ITupleBufferAccessor accessor = tupleMemoryManager.getTupleAccessor();
+ ITupleBufferAccessor accessor = tupleMemoryManager.getTupleBufferAccessor();
for (Map.Entry<TuplePointer, Integer> entry : mapInserted.entrySet()) {
accessor.reset(entry.getKey());
int dataLength = map.get(entry.getValue());
@@ -207,8 +207,7 @@
}
private ByteBuffer deleteRandomSelectedTuples(Map<Integer, Integer> map, Map<TuplePointer, Integer> mapInserted,
- int minNumOfRecordTobeDeleted)
- throws HyracksDataException {
+ int minNumOfRecordTobeDeleted) throws HyracksDataException {
ByteBuffer buffer = ByteBuffer.allocate(Common.BUDGET);
FixedSizeFrame frame = new FixedSizeFrame(buffer);
FrameTupleAppender appender = new FrameTupleAppender();
@@ -217,7 +216,7 @@
assert (minNumOfRecordTobeDeleted < mapInserted.size());
int countDeleted = minNumOfRecordTobeDeleted + random.nextInt(mapInserted.size() - minNumOfRecordTobeDeleted);
- ITupleBufferAccessor accessor = tupleMemoryManager.getTupleAccessor();
+ ITupleBufferAccessor accessor = tupleMemoryManager.getTupleBufferAccessor();
for (int i = 0; i < countDeleted; i++) {
Iterator<Map.Entry<TuplePointer, Integer>> iter = mapInserted.entrySet().iterator();
assert (iter.hasNext());
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
index bb8627a..21c5215 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
@@ -42,7 +42,7 @@
import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
import org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
-import org.apache.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.connectors.MToNBroadcastConnectorDescriptor;
import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
import org.apache.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -124,7 +124,7 @@
IConnectorDescriptor conn4 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn4, sorter2, 0, group2, 0);
- IConnectorDescriptor conn5 = new MToNReplicatingConnectorDescriptor(spec);
+ IConnectorDescriptor conn5 = new MToNBroadcastConnectorDescriptor(spec);
spec.connect(conn5, group2, 0, printer, 0);
spec.addRoot(printer);
@@ -197,7 +197,7 @@
IConnectorDescriptor conn4 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn4, sorter2, 0, group2, 0);
- IConnectorDescriptor conn5 = new MToNReplicatingConnectorDescriptor(spec);
+ IConnectorDescriptor conn5 = new MToNBroadcastConnectorDescriptor(spec);
spec.connect(conn5, group2, 0, printer, 0);
spec.addRoot(printer);
@@ -270,7 +270,7 @@
IConnectorDescriptor conn4 = new OneToOneConnectorDescriptor(spec);
spec.connect(conn4, sorter2, 0, group2, 0);
- IConnectorDescriptor conn5 = new MToNReplicatingConnectorDescriptor(spec);
+ IConnectorDescriptor conn5 = new MToNBroadcastConnectorDescriptor(spec);
spec.connect(conn5, group2, 0, printer, 0);
spec.addRoot(printer);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
index a9cf4ad..49b6731 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
@@ -41,7 +41,7 @@
import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
import org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory;
import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
-import org.apache.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.connectors.MToNBroadcastConnectorDescriptor;
import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
import org.apache.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -655,7 +655,7 @@
.of(UTF8StringPointable.FACTORY) }));
spec.connect(custJoinConn, custScanner, 0, join, 1);
- IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
+ IConnectorDescriptor joinPrinterConn = new MToNBroadcastConnectorDescriptor(spec);
spec.connect(joinPrinterConn, join, 0, printer, 0);
spec.addRoot(printer);
@@ -746,7 +746,7 @@
.of(UTF8StringPointable.FACTORY) }));
spec.connect(custJoinConn, custScanner, 0, join, 1);
- IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
+ IConnectorDescriptor joinPrinterConn = new MToNBroadcastConnectorDescriptor(spec);
spec.connect(joinPrinterConn, join, 0, printer, 0);
spec.addRoot(printer);
@@ -837,7 +837,7 @@
.of(UTF8StringPointable.FACTORY) }));
spec.connect(custJoinConn, custScanner, 0, join, 1);
- IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
+ IConnectorDescriptor joinPrinterConn = new MToNBroadcastConnectorDescriptor(spec);
spec.connect(joinPrinterConn, join, 0, printer, 0);
spec.addRoot(printer);
@@ -924,7 +924,7 @@
.of(UTF8StringPointable.FACTORY) }));
spec.connect(custJoinConn, custScanner, 0, join, 1);
- IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
+ IConnectorDescriptor joinPrinterConn = new MToNBroadcastConnectorDescriptor(spec);
spec.connect(joinPrinterConn, join, 0, printer, 0);
spec.addRoot(printer);
@@ -1023,7 +1023,7 @@
IConnectorDescriptor custJoinConn = new OneToOneConnectorDescriptor(spec);
spec.connect(custJoinConn, custMat, 0, join, 1);
- IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
+ IConnectorDescriptor joinPrinterConn = new MToNBroadcastConnectorDescriptor(spec);
spec.connect(joinPrinterConn, join, 0, printer, 0);
spec.addRoot(printer);
diff --git a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
index 039d936..0c68273 100644
--- a/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
+++ b/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
@@ -43,7 +43,7 @@
import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory;
import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import org.apache.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.connectors.MToNBroadcastConnectorDescriptor;
import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider;
import org.apache.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory;
@@ -189,7 +189,7 @@
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
spec.connect(ordJoinConn, ordScanner, 0, join, 0);
- IConnectorDescriptor custJoinConn = new MToNReplicatingConnectorDescriptor(spec);
+ IConnectorDescriptor custJoinConn = new MToNBroadcastConnectorDescriptor(spec);
spec.connect(custJoinConn, custScanner, 0, join, 1);
IConnectorDescriptor joinPrinterConn = new OneToOneConnectorDescriptor(spec);
@@ -266,10 +266,10 @@
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
spec.connect(ordJoinConn, ordScanner, 0, join, 0);
- IConnectorDescriptor custJoinConn = new MToNReplicatingConnectorDescriptor(spec);
+ IConnectorDescriptor custJoinConn = new MToNBroadcastConnectorDescriptor(spec);
spec.connect(custJoinConn, custScanner, 0, join, 1);
- IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
+ IConnectorDescriptor joinPrinterConn = new MToNBroadcastConnectorDescriptor(spec);
spec.connect(joinPrinterConn, join, 0, printer, 0);
spec.addRoot(printer);
@@ -343,10 +343,10 @@
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
spec.connect(ordJoinConn, ordScanner, 0, join, 0);
- IConnectorDescriptor custJoinConn = new MToNReplicatingConnectorDescriptor(spec);
+ IConnectorDescriptor custJoinConn = new MToNBroadcastConnectorDescriptor(spec);
spec.connect(custJoinConn, custScanner, 0, join, 1);
- IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
+ IConnectorDescriptor joinPrinterConn = new MToNBroadcastConnectorDescriptor(spec);
spec.connect(joinPrinterConn, join, 0, printer, 0);
spec.addRoot(printer);
@@ -425,10 +425,10 @@
IConnectorDescriptor ordJoinConn = new OneToOneConnectorDescriptor(spec);
spec.connect(ordJoinConn, ordScanner, 0, join, 0);
- IConnectorDescriptor custJoinConn = new MToNReplicatingConnectorDescriptor(spec);
+ IConnectorDescriptor custJoinConn = new MToNBroadcastConnectorDescriptor(spec);
spec.connect(custJoinConn, custScanner, 0, join, 1);
- IConnectorDescriptor joinPrinterConn = new MToNReplicatingConnectorDescriptor(spec);
+ IConnectorDescriptor joinPrinterConn = new MToNBroadcastConnectorDescriptor(spec);
spec.connect(joinPrinterConn, join, 0, printer, 0);
spec.addRoot(printer);
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java b/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
index 3b71be8..86efdf8 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
@@ -27,7 +27,6 @@
import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
import org.apache.hyracks.data.std.primitive.IntegerPointable;
import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
-import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProvider;
import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleReference;
import org.apache.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
@@ -130,7 +129,8 @@
if (c == 0) {
return i;
} else {
- int pageId = IntegerPointable.getInteger(frameTuple.getFieldData(cmp.getKeyFieldCount() - 1), getChildPointerOff(frameTuple));
+ int pageId = IntegerPointable.getInteger(frameTuple.getFieldData(cmp.getKeyFieldCount() - 1),
+ getChildPointerOff(frameTuple));
traverseList.add(pageId, -1, parentIndex);
}
}
@@ -205,9 +205,9 @@
protected int pointerCmp(ITupleReference tupleA, ITupleReference tupleB, MultiComparator cmp)
throws HyracksDataException {
- return childPtrCmp
- .compare(tupleA.getFieldData(cmp.getKeyFieldCount() - 1), getChildPointerOff(tupleA), childPtrSize,
- tupleB.getFieldData(cmp.getKeyFieldCount() - 1), getChildPointerOff(tupleB), childPtrSize);
+ return childPtrCmp.compare(tupleA.getFieldData(cmp.getKeyFieldCount() - 1), getChildPointerOff(tupleA),
+ childPtrSize, tupleB.getFieldData(cmp.getKeyFieldCount() - 1), getChildPointerOff(tupleB),
+ childPtrSize);
}
public int getTupleSize(ITupleReference tuple) {
@@ -284,8 +284,9 @@
for (int i = 0; i < tupleCount; i++) {
int tupleOff = slotManager.getTupleOff(slotManager.getSlotOff(i));
frameTuple.resetByTupleOffset(buf, tupleOff);
- int intVal = IntegerPointable.getInteger(buf.array(), frameTuple.getFieldStart(frameTuple.getFieldCount() - 1)
- + frameTuple.getFieldLength(frameTuple.getFieldCount() - 1));
+ int intVal = IntegerPointable.getInteger(buf.array(),
+ frameTuple.getFieldStart(frameTuple.getFieldCount() - 1)
+ + frameTuple.getFieldLength(frameTuple.getFieldCount() - 1));
ret.add(intVal);
}
return ret;
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java b/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
index e4df88a..8064084 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
@@ -20,7 +20,6 @@
import org.apache.hyracks.api.dataflow.value.ILinearizeComparator;
import org.apache.hyracks.data.std.primitive.DoublePointable;
-import org.apache.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProvider;
import org.apache.hyracks.storage.am.common.ophelpers.DoubleArrayList;
import org.apache.hyracks.storage.am.common.ophelpers.IntArrayList;
@@ -32,13 +31,13 @@
* number generics in Java) in the two-dimensional space. For more dimensions, the
* state machine has to be automatically generated. The idea of the fractal generation
* of the curve is described e.g. in http://dl.acm.org/ft_gateway.cfm?id=383528&type=pdf
- *
- * Unlike the described approach, this comparator does not compute the hilbert value at
+ *
+ * Unlike the described approach, this comparator does not compute the hilbert value at
* any point. Instead, it only evaluates how the two inputs compare to each other. This
* is done by starting at the lowest hilbert resolution and zooming in on the fractal until
* the two points are in different quadrants.
- *
- * As a performance optimization, the state of the state machine is saved in a stack and
+ *
+ * As a performance optimization, the state of the state machine is saved in a stack and
* maintained over comparisons. The idea behind this is that comparisons are usually in a
* similar area (e.g. geo coordinates). Zooming in from [-MAX_VALUE, MAX_VALUE] would take
* ~300 steps every time. Instead, the comparator start from the previous state and zooms out
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java b/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
index 0280ba5..344431f 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
@@ -20,7 +20,6 @@
import org.apache.hyracks.api.dataflow.value.ILinearizeComparator;
import org.apache.hyracks.data.std.primitive.DoublePointable;
-import org.apache.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
import org.apache.hyracks.storage.am.common.api.IPrimitiveValueProvider;
import org.apache.hyracks.storage.am.common.ophelpers.DoubleArrayList;
import org.apache.hyracks.storage.am.rtree.impls.DoublePrimitiveValueProviderFactory;
diff --git a/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java b/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
index f62ae27..4e0a17a 100644
--- a/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
+++ b/hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
@@ -21,7 +21,6 @@
import org.apache.hyracks.api.dataflow.value.ILinearizeComparator;
import org.apache.hyracks.data.std.primitive.DoublePointable;
import org.apache.hyracks.data.std.primitive.IntegerPointable;
-import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
import org.apache.hyracks.storage.am.common.ophelpers.DoubleArrayList;
/*
diff --git a/pom.xml b/pom.xml
index 8f00aac..a88bbaa 100644
--- a/pom.xml
+++ b/pom.xml
@@ -168,6 +168,7 @@
<exclude>**/src/main/resources/*.cleaned</exclude>
<exclude>**/ClusterControllerService/**</exclude>
<exclude>**/output/**</exclude>
+ <exclude>**/bin/**</exclude>
<exclude>**/*.iml</exclude>
</excludes>
</configuration>
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newchange
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 1
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 8:
Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/587/
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 8
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join operator and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join operator and updating the range connectors.
......................................................................
Patch Set 14:
Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/974/
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 14
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 8: Verified-1
Build Failed
https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/587/ : ABORTED
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 8
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join operator and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join operator and updating the range connectors.
......................................................................
Patch Set 13:
Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/972/
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 13
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updated the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updated the range connectors.
......................................................................
Patch Set 1: Verified-1
Build Failed
https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/512/ : FAILURE
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 1
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Preston Carman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,
I'd like you to reexamine a change. Please visit
https://asterix-gerrit.ics.uci.edu/456
to look at the new patch set (#9).
Change subject: Adding merge join and updating the range connectors.
......................................................................
Adding merge join and updating the range connectors.
Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
C algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
D algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
M algebricks/algebricks-data/pom.xml
M algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
M algebricks/algebricks-examples/piglet-example/pom.xml
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputer.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFactory.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFamily.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/storage/IGrowableIntArray.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTuplePairComparator.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
A hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/util/InputSplitsProxy.java
M hyracks/hyracks-dataflow-std/pom.xml
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameTupleBufferAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
M hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
M hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java
M hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/TreeIndexBufferCacheWarmup.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/PathList.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/DoubleArrayList.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/IntArrayList.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/LongArrayList.java
M pom.xml
73 files changed, 2,356 insertions(+), 566 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/56/456/9
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 9
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 3: -Verified
Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/519/
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 3
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 3: Verified+1
Build Successful
https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/516/ : SUCCESS
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 3
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join operator and updating the range connectors.
Posted by "Preston Carman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,
I'd like you to reexamine a change. Please visit
https://asterix-gerrit.ics.uci.edu/456
to look at the new patch set (#15).
Change subject: Adding merge join operator and updating the range connectors.
......................................................................
Adding merge join operator and updating the range connectors.
The update is required by the interval join algorithms in asterixdb.
Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
C algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
D algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
M algebricks/algebricks-data/pom.xml
M algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/IBinaryRangeComparatorFactory.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputer.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFactory.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFamily.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/storage/IGrowableIntArray.java
M hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryComparatorFactory.java
A hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryRangeAscComparatorFactory.java
A hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/accessors/PointableBinaryRangeDescComparatorFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
A hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
M hyracks/hyracks-dataflow-std/pom.xml
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleBufferManager.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/TupleAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractFrameSorter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/structures/TuplePointer.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/util/FrameTuplePairComparator.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleMemoryManagerTest.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManagerTest.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableTupleMemoryManagerTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/pom.xml
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
A hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/FieldRangePartitionComputerFactoryTest.java
M hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java
D hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/LongArrayList.java
M hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/TreeIndexBufferCacheWarmup.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/PathList.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/DoubleArrayList.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/IntArrayList.java
C hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/LongArrayList.java
M pom.xml
83 files changed, 3,427 insertions(+), 654 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/56/456/15
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 15
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Change in hyracks[master]: Adding merge join operator and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join operator and updating the range connectors.
......................................................................
Patch Set 12:
Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/879/
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 12
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 2: Verified+1
Build Successful
https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/513/ : SUCCESS
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 2
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updated the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updated the range connectors.
......................................................................
Patch Set 1:
Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/512/
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 1
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 6: Verified+1
Build Successful
https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/544/ : SUCCESS
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 6
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 9: Verified+1
Build Successful
https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/588/ : SUCCESS
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 9
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Preston Carman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,
I'd like you to reexamine a change. Please visit
https://asterix-gerrit.ics.uci.edu/456
to look at the new patch set (#6).
Change subject: Adding merge join and updating the range connectors.
......................................................................
Adding merge join and updating the range connectors.
Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
M algebricks/algebricks-data/pom.xml
M algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
M algebricks/algebricks-examples/piglet-example/pom.xml
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
M hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTuplePairComparator.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFamily.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
A hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopHashTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopNewPartitionerTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopPartitionerTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/util/InputSplitsProxy.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/GroupingHashTable.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameTupleBufferAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/util/PartitionUtil.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
M pom.xml
77 files changed, 2,440 insertions(+), 515 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/56/456/6
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 6
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 10: Verified+1
Build Successful
https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/589/ : SUCCESS
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 10
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 7: Verified+1
Build Successful
https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/548/ : SUCCESS
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 7
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join operator and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join operator and updating the range connectors.
......................................................................
Patch Set 15: Verified+1
Build Successful
https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/991/ : SUCCESS
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 15
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Preston Carman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,
I'd like you to reexamine a change. Please visit
https://asterix-gerrit.ics.uci.edu/456
to look at the new patch set (#2).
Change subject: Adding merge join and updating the range connectors.
......................................................................
Adding merge join and updating the range connectors.
Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
M algebricks/algebricks-examples/piglet-example/pom.xml
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
M hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java
M hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/ByteArrayPointableTest.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTuplePairComparator.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFamily.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
A hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopHashTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopNewPartitionerTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopPartitionerTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/util/InputSplitsProxy.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/GroupingHashTable.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameTupleBufferAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/util/PartitionUtil.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
M pom.xml
74 files changed, 2,315 insertions(+), 457 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/56/456/2
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 2
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 4:
Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/542/
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 10:
Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/589/
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 10
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join operator and updating the range connectors.
Posted by "Preston Carman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,
I'd like you to reexamine a change. Please visit
https://asterix-gerrit.ics.uci.edu/456
to look at the new patch set (#13).
Change subject: Adding merge join operator and updating the range connectors.
......................................................................
Adding merge join operator and updating the range connectors.
The update is required by the interval join algorithms in asterixdb.
Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
C algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
D algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
M algebricks/algebricks-data/pom.xml
M algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputer.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFactory.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFamily.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/storage/IGrowableIntArray.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
A hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
M hyracks/hyracks-dataflow-std/pom.xml
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleBufferManager.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/TupleAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractFrameSorter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/util/FrameTuplePairComparator.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleMemoryManagerTest.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManagerTest.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableTupleMemoryManagerTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
M hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java
D hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/LongArrayList.java
M hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/TreeIndexBufferCacheWarmup.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/PathList.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/DoubleArrayList.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/IntArrayList.java
C hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/LongArrayList.java
M pom.xml
76 files changed, 2,658 insertions(+), 549 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/56/456/13
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 13
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 7:
Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/548/
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 7
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join operator and updating the range connectors.
Posted by "Ian Maxon (Code Review)" <do...@asterixdb.incubator.apache.org>.
Ian Maxon has posted comments on this change.
Change subject: Adding merge join operator and updating the range connectors.
......................................................................
Patch Set 15: Verified-1
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 15
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join operator and updating the range connectors.
Posted by "Preston Carman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,
I'd like you to reexamine a change. Please visit
https://asterix-gerrit.ics.uci.edu/456
to look at the new patch set (#14).
Change subject: Adding merge join operator and updating the range connectors.
......................................................................
Adding merge join operator and updating the range connectors.
The update is required by the interval join algorithms in asterixdb.
Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
C algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
D algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
M algebricks/algebricks-data/pom.xml
M algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputer.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFactory.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFamily.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/storage/IGrowableIntArray.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
A hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
M hyracks/hyracks-dataflow-std/pom.xml
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/IPartitionedTupleBufferManager.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/ITupleBufferManager.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/TupleAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VariableDeletableTupleMemoryManager.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractFrameSorter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/util/DeletableFrameTupleAppender.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/util/FrameTuplePairComparator.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/AbstractTupleMemoryManagerTest.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManagerTest.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/buffermanager/VariableTupleMemoryManagerTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
M hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java
D hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/LongArrayList.java
M hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/TreeIndexBufferCacheWarmup.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/PathList.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/DoubleArrayList.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/IntArrayList.java
C hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/LongArrayList.java
M pom.xml
76 files changed, 2,662 insertions(+), 550 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/56/456/14
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 14
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Preston Carman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,
I'd like you to reexamine a change. Please visit
https://asterix-gerrit.ics.uci.edu/456
to look at the new patch set (#5).
Change subject: Adding merge join and updating the range connectors.
......................................................................
Adding merge join and updating the range connectors.
Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
M algebricks/algebricks-data/pom.xml
M algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
M algebricks/algebricks-examples/piglet-example/pom.xml
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
M hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTuplePairComparator.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFamily.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
A hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopHashTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopNewPartitionerTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopPartitionerTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/util/InputSplitsProxy.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/GroupingHashTable.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameTupleBufferAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/util/PartitionUtil.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
M pom.xml
77 files changed, 2,409 insertions(+), 487 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/56/456/5
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 5
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Preston Carman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,
I'd like you to reexamine a change. Please visit
https://asterix-gerrit.ics.uci.edu/456
to look at the new patch set (#3).
Change subject: Adding merge join and updating the range connectors.
......................................................................
Adding merge join and updating the range connectors.
Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
M algebricks/algebricks-examples/piglet-example/pom.xml
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
M hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java
M hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/ByteArrayPointableTest.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTuplePairComparator.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFamily.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
A hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopHashTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopNewPartitionerTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopPartitionerTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/util/InputSplitsProxy.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/GroupingHashTable.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameTupleBufferAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/util/PartitionUtil.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
M pom.xml
74 files changed, 2,319 insertions(+), 459 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/56/456/3
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 3
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 4: Verified-1
Build Failed
https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/542/ : FAILURE
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join operator and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join operator and updating the range connectors.
......................................................................
Patch Set 13: Verified-1
Build Failed
https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/972/ : FAILURE
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 13
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Preston Carman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,
I'd like you to reexamine a change. Please visit
https://asterix-gerrit.ics.uci.edu/456
to look at the new patch set (#11).
Change subject: Adding merge join and updating the range connectors.
......................................................................
Adding merge join and updating the range connectors.
Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractPropagatePropertiesForUsedVariablesPOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
C algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
D algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
M algebricks/algebricks-data/pom.xml
M algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputer.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFactory.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFamily.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/storage/IGrowableIntArray.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
A hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
M hyracks/hyracks-dataflow-std/pom.xml
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractFrameSorter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
M hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java
M hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/TreeIndexBufferCacheWarmup.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/PathList.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/DoubleArrayList.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/IntArrayList.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/LongArrayList.java
M pom.xml
63 files changed, 2,229 insertions(+), 390 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/56/456/11
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 11
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Change in hyracks[master]: Adding merge join operator and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join operator and updating the range connectors.
......................................................................
Patch Set 15:
Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/991/
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 15
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 2:
Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/513/
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 2
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join operator and updating the range connectors.
Posted by "Preston Carman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Preston Carman has abandoned this change.
Change subject: Adding merge join operator and updating the range connectors.
......................................................................
Abandoned
Moved to new repository.
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: abandon
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 15
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 3:
Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/516/
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 3
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 11: Verified+1
Build Successful
https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/878/ : SUCCESS
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 11
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 5:
Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/543/
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 5
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join operator and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join operator and updating the range connectors.
......................................................................
Patch Set 12: Verified+1
Build Successful
https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/879/ : SUCCESS
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 12
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 7:
Can we let ITuplePartitionComputer stay as it is and instead you add a new partition computer interface for this 1-tuple-to-many-partition case?
It seems do-able to me if you implement a new AbstractConnectorDescriptor for this specialized connector. At conceptual level, the new connector is different from M-To-N-Hash or M-To-N-Hash-Merge, so it deserves a new implementation.
In this way, we can avoid adding the per-tuple overhead (will be explained next) into the current joins/group-bys/connectors and would not affect the performance of usual group-by/join queries.
The current added per-tuple overheads include:
1. per-tuple Java Object creation overhead. For example, map.add(i) will wrap i as a Java Integer first though i is a primitive int. Then, in the runtime, the number of created Objects will be proportional to the number of data tuples. Therefore, it would be better to use a primitive int array to hold the partition ids and you can wrap that array as a class to do dynamical re-sizing.
2. The overhead for maintaining the partition id array, e.g., the count of partition ids. This should be small but is added to each tuple. This is un-avoidable for interval joins, but is avoidable for usual joins/groups.
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 7
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 5: Verified+1
Build Successful
https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/543/ : SUCCESS
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 5
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Preston Carman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,
I'd like you to reexamine a change. Please visit
https://asterix-gerrit.ics.uci.edu/456
to look at the new patch set (#8).
Change subject: Adding merge join and updating the range connectors.
......................................................................
Adding merge join and updating the range connectors.
Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
C algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
D algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
M algebricks/algebricks-data/pom.xml
M algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
M algebricks/algebricks-examples/piglet-example/pom.xml
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputer.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFactory.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFamily.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/storage/IGrowableIntArray.java
A hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/GrowableIntArray.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTuplePairComparator.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
A hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/util/InputSplitsProxy.java
M hyracks/hyracks-dataflow-std/pom.xml
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameTupleBufferAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
M hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
M hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java
M hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/TreeIndexBufferCacheWarmup.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/PathList.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/DoubleArrayList.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/IntArrayList.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/LongArrayList.java
M pom.xml
74 files changed, 2,449 insertions(+), 566 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/56/456/8
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 8
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Change in hyracks[master]: Adding merge join operator and updating the range connectors.
Posted by "Preston Carman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,
I'd like you to reexamine a change. Please visit
https://asterix-gerrit.ics.uci.edu/456
to look at the new patch set (#12).
Change subject: Adding merge join operator and updating the range connectors.
......................................................................
Adding merge join operator and updating the range connectors.
The update is required by the interval join algorithms in asterixdb.
Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractPropagatePropertiesForUsedVariablesPOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
C algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
D algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
M algebricks/algebricks-data/pom.xml
M algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/util/JoinUtils.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputer.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFactory.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFamily.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/storage/IGrowableIntArray.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
A hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopMapperOperatorDescriptor.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/HadoopReadOperatorDescriptor.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/mapreduce/MapperOperatorDescriptor.java
M hyracks/hyracks-dataflow-std/pom.xml
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/VPartitionTupleBufferManager.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractFrameSorter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/FrameSorterMergeSort.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
M hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java
D hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/ophelpers/LongArrayList.java
M hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/TreeIndexBufferCacheWarmup.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/PathList.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/DoubleArrayList.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/IntArrayList.java
C hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/LongArrayList.java
M pom.xml
64 files changed, 2,296 insertions(+), 568 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/56/456/12
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 12
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 3: Verified+1
Build Successful
https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/519/ : SUCCESS
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 3
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Preston Carman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,
I'd like you to reexamine a change. Please visit
https://asterix-gerrit.ics.uci.edu/456
to look at the new patch set (#4).
Change subject: Adding merge join and updating the range connectors.
......................................................................
Adding merge join and updating the range connectors.
Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
M algebricks/algebricks-data/pom.xml
M algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
M algebricks/algebricks-examples/piglet-example/pom.xml
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
M hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java
M hyracks/hyracks-data/hyracks-data-std/src/test/java/org/apache/hyracks/data/std/primitive/ByteArrayPointableTest.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTuplePairComparator.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFamily.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
A hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopHashTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopNewPartitionerTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopPartitionerTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/util/InputSplitsProxy.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/GroupingHashTable.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameTupleBufferAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/util/PartitionUtil.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
M pom.xml
78 files changed, 2,434 insertions(+), 503 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/56/456/4
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 6:
Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/544/
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 6
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 9:
Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/588/
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 9
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Preston Carman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,
I'd like you to reexamine a change. Please visit
https://asterix-gerrit.ics.uci.edu/456
to look at the new patch set (#10).
Change subject: Adding merge join and updating the range connectors.
......................................................................
Adding merge join and updating the range connectors.
Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
C algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
D algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
M algebricks/algebricks-data/pom.xml
M algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
M algebricks/algebricks-examples/piglet-example/pom.xml
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputer.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFactory.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITupleRangePartitionComputerFamily.java
A hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/storage/IGrowableIntArray.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTuplePairComparator.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
A hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/util/InputSplitsProxy.java
M hyracks/hyracks-dataflow-std/pom.xml
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameTupleBufferAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
M hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTree.java
M hyracks/hyracks-storage-am-btree/src/main/java/org/apache/hyracks/storage/am/btree/impls/BTreeOpContext.java
M hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/util/TreeIndexBufferCacheWarmup.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/impls/PathList.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/DoubleArrayList.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/IntArrayList.java
R hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/arraylist/LongArrayList.java
M pom.xml
74 files changed, 2,371 insertions(+), 581 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/56/456/10
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 10
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Preston Carman (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,
I'd like you to reexamine a change. Please visit
https://asterix-gerrit.ics.uci.edu/456
to look at the new patch set (#7).
Change subject: Adding merge join and updating the range connectors.
......................................................................
Adding merge join and updating the range connectors.
Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/IndexedNLJoinExpressionAnnotation.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BroadcastExchangePOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/MergeJoinPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomMergeExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RandomPartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SortMergeExchangePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
M algebricks/algebricks-data/pom.xml
M algebricks/algebricks-data/src/main/java/org/apache/hyracks/algebricks/data/IBinaryComparatorFactoryProvider.java
M algebricks/algebricks-examples/piglet-example/pom.xml
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ConsolidateSelectsRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractFunctionsFromJoinConditionRule.java
M hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/comm/io/FrameTuplePairComparator.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFamily.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFamily.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
A hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangePartitionType.java
M hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopHashTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopNewPartitionerTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopPartitionerTuplePartitionComputerFactory.java
M hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/util/InputSplitsProxy.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/AbstractPartitionDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwarePartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitionMergingConnectorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNRangePartitioningConnectorDescriptor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionRangeDataWriter.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/HashSpillableTableFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldGroupAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/AvgFieldMergeAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/CountFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/FloatSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/aggregators/IntSumFieldAggregatorFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/group/hash/GroupingHashTable.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/IMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoin.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinLocks.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoinOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeJoiner.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/MergeStatus.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinChecker.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/NaturalMergeJoinCheckerFactory.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoinOperatorDescriptor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/ExternalSortRunMerger.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/TupleSorterHeapSort.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/IFrameTupleBufferAccessor.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/ITupleBufferManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManager.java
M hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
A hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/util/PartitionUtil.java
M hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/sort/buffermanager/VariableTupleMemoryManagerTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/CountOfCountsTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderHashJoinTest.java
M hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/TPCHCustomerOrderNestedLoopJoinTest.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/frames/RTreeNSMInteriorFrame.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/HilbertDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveDoubleComparator.java
M hyracks/hyracks-storage-am-rtree/src/main/java/org/apache/hyracks/storage/am/rtree/linearize/ZCurveIntComparator.java
M pom.xml
77 files changed, 2,440 insertions(+), 515 deletions(-)
git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/56/456/7
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 7
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Change in hyracks[master]: Adding merge join operator and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join operator and updating the range connectors.
......................................................................
Patch Set 14: Verified+1
Build Successful
https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/974/ : SUCCESS
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 14
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No
Change in hyracks[master]: Adding merge join and updating the range connectors.
Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.
Change subject: Adding merge join and updating the range connectors.
......................................................................
Patch Set 11:
Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/878/
--
To view, visit https://asterix-gerrit.ics.uci.edu/456
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings
Gerrit-MessageType: comment
Gerrit-Change-Id: I9b0c6c4e73bbaa17f1477c2feb28aa293a18fa35
Gerrit-PatchSet: 11
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Preston Carman <pr...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No