You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org> on 2015/10/02 08:58:29 UTC

Change in hyracks[master]: PLEASE EDIT to provide a meaningful commit message!

Wenhai Li has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/427

Change subject: PLEASE EDIT to provide a meaningful commit message!
......................................................................

PLEASE EDIT to provide a meaningful commit message!

The following commits from your working branch will be included:
exchange optimization for lookup and delete when the operation is merely directed by the primary key.

commit 8d803cab358382b7c1510900cc7551de5fcc8480
Author: Michael <lw...@yahoo.com>
Date:   Thu Oct 1 23:58:48 2015 -0700

    exchange optimization for lookup and delete

Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
M algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
3 files changed, 130 insertions(+), 14 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/27/427/1

diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
index 53857d2..9559e6e 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
@@ -75,6 +75,11 @@
             LogicalVariable payLoadVar, List<LogicalVariable> additionalNonKeyFields, RecordDescriptor recordDesc,
             JobGenContext context, JobSpecification jobSpec) throws AlgebricksException;
 
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(IDataSource<S> dataSource,
+            IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv, List<LogicalVariable> keys,
+            LogicalVariable payLoadVar, List<LogicalVariable> additionalNonKeyFields, RecordDescriptor recordDesc,
+            JobGenContext context, JobSpecification jobSpec, int nodeId) throws AlgebricksException;
+
     /**
      * Creates the insert runtime of IndexInsertDeletePOperator, which models
      * insert/delete operations into a secondary index.
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
index c774f44..ccf8c63 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
@@ -18,33 +18,50 @@
  */
 package org.apache.hyracks.algebricks.core.algebra.operators.physical;
 
+import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 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.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 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.expressions.ScalarFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator.Kind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 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.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.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.dataflow.IOperatorDescriptor;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
+import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.job.JobSpecification;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 
 @SuppressWarnings("rawtypes")
 public class InsertDeletePOperator extends AbstractPhysicalOperator {
@@ -71,6 +88,91 @@
     public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
         AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
         deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+    }
+
+    private int extractPrimaryKeyNodeId(IHyracksJobBuilder builder, JobGenContext context,
+            IOperatorSchema[] inputSchemas, ILogicalOperator op) throws AlgebricksException {
+        ILogicalOperator pop = op;
+        List<LogicalVariable> lv = new ArrayList<LogicalVariable>();
+        List<Mutable<ILogicalExpression>> le = null;
+        boolean found = false;
+        int route = -1;
+        while (pop.hasInputs()) {
+            pop = pop.getInputs().get(0).getValue();
+            if (pop.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                VariableUtilities.getProducedVariables(pop, lv);
+                le = ((AssignOperator) pop).getExpressions();
+                if (lv.size() != keys.size())
+                    continue;
+                boolean isPrimaryKey = true;
+                for (int i = 0; i < keys.size(); i++) {
+                    if (keys.get(i).getId() != lv.get(i).getId())
+                        isPrimaryKey = false;
+                }
+                if (isPrimaryKey) {
+                    found = true;
+                    break;
+                }
+            }
+        }
+        if (found) {
+            int keySize = 0;
+            if (((InsertDeleteOperator) op).getOperation() == Kind.DELETE) {
+                ILogicalOperator cop = null;
+                while (pop.hasInputs()) {
+                    pop = pop.getInputs().get(0).getValue();
+                    if (pop.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                        cop = pop;
+                    }
+                }
+                pop = cop;
+                lv = ((AssignOperator) pop).getVariables();
+                le = ((AssignOperator) pop).getExpressions();
+                keySize = lv.size() / 2;
+            }
+            boolean isEqual = true;
+            for (int i = 0; i < le.size() / 2; i++) {
+                if (le.get(i).getValue() != le.get(i + le.size() / 2).getValue())
+                    isEqual = false;
+            }
+            if (isEqual && keySize > 0) {
+                int nPartition = ((AlgebricksAbsolutePartitionConstraint) (context.getClusterLocations()))
+                        .getLocations().length;
+                IBinaryHashFunctionFactory[] hashFuns = JobGenHelper.variablesToBinaryHashFunctionFactories(lv,
+                        context.getTypeEnvironment(pop), context);
+                int h = 0;
+                for (int i = 0; i < keySize; i++) {
+                    IBinaryHashFunction bhf = hashFuns[i].createBinaryHashFunction();
+                    ILogicalExpression expr = le.get(i).getValue();
+                    IScalarEvaluatorFactory ief = null;
+                    if (LogicalExpressionTag.FUNCTION_CALL == expr.getExpressionTag())
+                        ief = context.getExpressionRuntimeProvider().createEvaluatorFactory(
+                                ((ScalarFunctionCallExpression) expr).getArguments().get(1).getValue(),
+                                context.getTypeEnvironment(op), inputSchemas, context);
+                    else
+                        ief = context.getExpressionRuntimeProvider().createEvaluatorFactory(expr,
+                                context.getTypeEnvironment(op), inputSchemas, context);
+                    ByteBufferInputStream buffer = new ByteBufferInputStream();
+                    IScalarEvaluator eval = ief.createScalarEvaluator(null);
+                    IPointable p = VoidPointable.FACTORY.createPointable();
+                    eval.evaluate(null, p);
+                    buffer.setByteBuffer(ByteBuffer.wrap(p.getByteArray(), p.getStartOffset(), p.getLength()), 0);
+                    int fh;
+                    try {
+                        fh = bhf.hash(p.getByteArray(), p.getStartOffset(), p.getLength());
+                        h = h * 31 + fh;
+                        buffer.close();
+                    } catch (IOException e) {
+                        throw new AlgebricksException("Extract primary key metadata error: " + pop.toString());
+                    }
+                }
+                if (h < 0)
+                    h = -(h + 1);
+                route = h % nPartition;
+            }
+        }
+
+        return route;
     }
 
     @Override
@@ -103,8 +205,13 @@
             runtimeAndConstraints = mp.getInsertRuntime(dataSource, propagatedSchema, typeEnv, keys, payload,
                     additionalFilteringKeys, inputDesc, context, spec, false);
         } else {
-            runtimeAndConstraints = mp.getDeleteRuntime(dataSource, propagatedSchema, typeEnv, keys, payload,
-                    additionalFilteringKeys, inputDesc, context, spec);
+            int nodeId = -1;
+            if (-1 != (nodeId = extractPrimaryKeyNodeId(builder, context, inputSchemas, op)))
+                runtimeAndConstraints = mp.getDeleteRuntime(dataSource, propagatedSchema, typeEnv, keys, payload,
+                        additionalFilteringKeys, inputDesc, context, spec, nodeId);
+            else
+                runtimeAndConstraints = mp.getDeleteRuntime(dataSource, propagatedSchema, typeEnv, keys, payload,
+                        additionalFilteringKeys, inputDesc, context, spec);
         }
 
         builder.contributeHyracksOperator(insertDeleteOp, runtimeAndConstraints.first);
diff --git a/algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java b/algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
index 7d9b3db..356b38d 100644
--- a/algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
+++ b/algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
@@ -178,15 +178,11 @@
 
     @Override
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertRuntime(
-            IDataSourceIndex<String, String> dataSource,
-            IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,
-            IVariableTypeEnvironment typeEnv,
-            List<LogicalVariable> primaryKeys,
-            List<LogicalVariable> secondaryKeys,
-            List<LogicalVariable> additionalNonKeyFields,
-            ILogicalExpression filterExpr,
-            RecordDescriptor recordDesc, JobGenContext context,
-            JobSpecification spec, boolean bulkload) throws AlgebricksException {
+            IDataSourceIndex<String, String> dataSource, IOperatorSchema propagatedSchema,
+            IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
+            List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalNonKeyFields,
+            ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec,
+            boolean bulkload) throws AlgebricksException {
         // TODO Auto-generated method stub
         return null;
     }
@@ -204,9 +200,9 @@
 
     @Override
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getTokenizerRuntime(
-            IDataSourceIndex<String, String> dataSource, IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,
-            IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
-            ILogicalExpression filterExpr, RecordDescriptor recordDesc,
+            IDataSourceIndex<String, String> dataSource, IOperatorSchema propagatedSchema,
+            IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
+            List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
             JobGenContext context, JobSpecification spec, boolean bulkload) throws AlgebricksException {
         // TODO Auto-generated method stub
         return null;
@@ -235,4 +231,12 @@
         return null;
     }
 
+    @Override
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(IDataSource<String> dataSource,
+            IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv, List<LogicalVariable> keys,
+            LogicalVariable payLoadVar, List<LogicalVariable> additionalNonKeyFields, RecordDescriptor recordDesc,
+            JobGenContext context, JobSpecification jobSpec, int nodeId) throws AlgebricksException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 }

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 1
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Steven Jacobs, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/427

to look at the new patch set (#11).

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query
......................................................................

Equivalence induced onetoone operatiions for insert/delete/upsert/query

commit 8d803cab358382b7c1510900cc7551de5fcc8480
Author: Michael <lw...@yahoo.com>
Date:   Thu Oct 1 23:58:48 2015 -0700

    exchange optimization for lookup and delete

Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionListVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.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/RandomPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
M algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
16 files changed, 497 insertions(+), 80 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/27/427/11
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 11
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................


Patch Set 9:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/888/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 9
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Exchange optimization for lookup and delete

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Exchange optimization for lookup and delete
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/483/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 2
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Wenhai Li (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/427

to look at the new patch set (#10).

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................

Equivalence induced onetoone operatiions for insert/delete/upsert/query
exchange optimization for lookup and delete when the operation is merely directed by the primary key.

commit 8d803cab358382b7c1510900cc7551de5fcc8480
Author: Michael <lw...@yahoo.com>
Date:   Thu Oct 1 23:58:48 2015 -0700

    exchange optimization for lookup and delete

Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.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/RandomPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
M algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
16 files changed, 455 insertions(+), 78 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/27/427/10
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 10
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query
......................................................................


Patch Set 12:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/966/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 12
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................


Patch Set 8:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/885/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 8
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query
......................................................................


Patch Set 11:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/956/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 11
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................


Patch Set 10:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/912/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 10
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Wenhai Li has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................


Patch Set 9:

Hi, steven.
Attached the examples. You can apply the upsert on that dataset as well. Just notice the output of the hyracks plan. You will get only one route in the constraint of each operator.

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 9
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Steven Jacobs, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/427

to look at the new patch set (#13).

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query
......................................................................

Equivalence induced onetoone operatiions for insert/delete/upsert/query

commit 8d803cab358382b7c1510900cc7551de5fcc8480
Author: Michael <lw...@yahoo.com>
Date:   Thu Oct 1 23:58:48 2015 -0700

    exchange optimization for lookup and delete

Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteUpsertOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionListVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/DefaultNodeGroupDomain.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/FileSplitDomain.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/INodeDomain.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/RandomPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/ResultSetDomain.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
M algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
21 files changed, 512 insertions(+), 87 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/27/427/13
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 13
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in hyracks[master]: Equivalence induced onetoone operatiions

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions
......................................................................


Patch Set 5:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/427/5/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java:

Line 73:     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(IDataSource<S> dataSource,
You can add the new parameter "hash value" into the two old methods. In this way, your implementation will not duplicate codes of the old methods.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 5
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: Yes

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................


Patch Set 8: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/885/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 8
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Wenhai Li (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/427

to look at the new patch set (#6).

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................

Equivalence induced onetoone operatiions for insert/delete/upsert/query
exchange optimization for lookup and delete when the operation is merely directed by the primary key.

commit 8d803cab358382b7c1510900cc7551de5fcc8480
Author: Michael <lw...@yahoo.com>
Date:   Thu Oct 1 23:58:48 2015 -0700

    exchange optimization for lookup and delete

Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.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/RandomPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
M algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
16 files changed, 422 insertions(+), 39 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/27/427/6
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 6
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in hyracks[master]: Equivalence induced onetoone operatiions

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions
......................................................................


Patch Set 4: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/536/ : ABORTED

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions
......................................................................


Patch Set 5:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/427/5/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionVisitor.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionVisitor.java:

Line 237:         varToExpr.put(vars, null);
> Why do you need to put vars-->null map into the varToExprMap?
Sorry, ignore that "Those variables are produced by the index-search function call, just that index-search is not a stateless function."  I thought that was in visitUnnestmap.

My confusion is that here you get the used variables, but other places get produced variables.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 5
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: Yes

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query
......................................................................


Patch Set 12: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/966/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 12
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................


Patch Set 6: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/883/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 6
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Wenhai Li (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/427

to look at the new patch set (#9).

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................

Equivalence induced onetoone operatiions for insert/delete/upsert/query
exchange optimization for lookup and delete when the operation is merely directed by the primary key.

commit 8d803cab358382b7c1510900cc7551de5fcc8480
Author: Michael <lw...@yahoo.com>
Date:   Thu Oct 1 23:58:48 2015 -0700

    exchange optimization for query, upsert, lookup and delete

drop dataverse twitter if exists;
create dataverse twitter;
use dataverse twitter;
create type TweetMessageType as closed {
        id: string,
        message-text: string
}
create type nest as closed{
nid:string?,
nested1 : TweetMessageType,
nested2 : TweetMessageType
}
create type doublenest as closed{
nested1:nest,
nested2:nest
}
create dataset doublenests(doublenest)
primary key nested1.nested1.id, nested1.nested2.id, nested2.nested2.message-text;

use dataverse twitter;
insert into dataset doublenests(
{
"nested2":{"nid":"mi","nested2":{"id":"ok","message-text":"hello"},"nested1":{"id":"ok","message-text":"hello"}},
"nested1":{"nested1":{"id":"ok","message-text":"hello"},"nested2":{"id":"ok","message-text":"hello"},"nid":"mi"}
});


use dataverse twitter;
insert into dataset doublenests(
{
"nested1":{"nid":"mi","nested1":{"id":"ok","message-text":"hello"},"nested2":{"id":"ok","message-text":"hello"}},
"nested2":{"nid":"mi","nested1":{"id":"ok","message-text":"hello"},"nested2":{"id":"ok","message-text":"hello"}}
});

use dataverse twitter;
delete $d from dataset doublenests
where $d.nested1.nested1.id = "ok" 
and $d.nested1.nested2.id = "ok" 
and $d.nested1.nested2.message-text = "hello" 
and $d.nested2.nested2.message-text = "hello"

use dataverse twitter;
for $d in dataset doublenests
where $d.nested1.nested1.id = "ok" 
and $d.nested1.nested2.id = "ok" 
and $d.nested1.nested2.message-text = "hello" 
and $d.nested2.nested2.message-text = "hello"
return $d

Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.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/RandomPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
M algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
16 files changed, 421 insertions(+), 38 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/27/427/9
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 9
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Steven Jacobs (Code Review)" <do...@asterixdb.incubator.apache.org>.
Steven Jacobs has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................


Patch Set 8:

(1 comment)

Hey, I was starting to look at this but it's unclear to me what the intention of the change is. Could you provide a brief example of the change?

https://asterix-gerrit.ics.uci.edu/#/c/427/8/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java:

Line 316:                     + ".\n");
Is this the correct formatting change?


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 8
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: Yes

Change in hyracks[master]: Equivalence induced onetoone operatiions

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions
......................................................................


Patch Set 5:

(10 comments)

https://asterix-gerrit.ics.uci.edu/#/c/427/5/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java:

Line 76:             JobGenContext context, JobSpecification jobSpec, boolean bulkload, int nodeId) throws AlgebricksException;
this should be a hash value (maybe it's only a naming issue),  the specific node id (partition) is determined within this method.
That could keep the physical optimizations unaware of the low level cluster configurations.


Line 86:             JobGenContext context, JobSpecification jobSpec, int nodeId) throws AlgebricksException;
same as above.


https://asterix-gerrit.ics.uci.edu/#/c/427/5/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteOperator.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteOperator.java:

Line 46:     private int partitionId;
that should be a hash value instead of the real partition id (maybe it's a naming issue).
Also these two properties shouldn't appear in a logical operator.  Instead, they should be in InsertDeletePOperator.


Line 116:         if (-1 != partition)
to be more readable: (partition != -1)


https://asterix-gerrit.ics.uci.edu/#/c/427/5/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java:

Line 87:     private boolean isSinglePartition;
Both these two variables shouldn't appear in this class.
We have a clear logical/physical separation in the compiler.  A logical operator shouldn't be aware of physical details.
Instead, the two should go to BTree Search.


https://asterix-gerrit.ics.uci.edu/#/c/427/5/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionVisitor.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionVisitor.java:

Line 64:  * @author michael
What's the purpose of this class?

If it is a general visitor for you to find the <var, exp> map produced by an operator, it seems the implementations of several other operators are missing,  e.g., aggregate, running aggregate, and group-by


Line 237:         varToExpr.put(vars, null);
Why do you need to put vars-->null map into the varToExprMap?
Those variables are produced by the index-search function call, just that index-search is not a stateless function.


https://asterix-gerrit.ics.uci.edu/#/c/427/5/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java:

Line 90:     public abstract void setSingle(boolean isSingle);
"abstract" is not needed because it is an interface.
The same for the rest methods.


https://asterix-gerrit.ics.uci.edu/#/c/427/5/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java:

Line 88:                 return null;
return new StructuralPropertiesVector(diffLocals, null);


https://asterix-gerrit.ics.uci.edu/#/c/427/5/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
File algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java:

Line 204:         /*if (required != null && required.getPartitioningProperty() != null && op.getInputs().size() == 1
remove commented code.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 5
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: Yes

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................


Patch Set 7:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/884/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 7
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Steven Jacobs (Code Review)" <do...@asterixdb.incubator.apache.org>.
Steven Jacobs has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................


Patch Set 10: Code-Review+2

This one is fine. The Asterix one still needs the +2 before the merge though.

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 10
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/537/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 5
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query
......................................................................


Patch Set 13:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/967/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 13
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions The following commi...

Posted by "Wenhai Li (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/427

to look at the new patch set (#3).

Change subject: Equivalence induced onetoone operatiions The following commits from your working branch will be included: exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................

Equivalence induced onetoone operatiions
The following commits from your working branch will be included:
exchange optimization for lookup and delete when the operation is merely directed by the primary key.

commit 8d803cab358382b7c1510900cc7551de5fcc8480
Author: Michael <lw...@yahoo.com>
Date:   Thu Oct 1 23:58:48 2015 -0700

    exchange optimization for lookup and delete

Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.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/RandomPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
M algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
15 files changed, 421 insertions(+), 19 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/27/427/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 3
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Steven Jacobs (Code Review)" <do...@asterixdb.incubator.apache.org>.
Steven Jacobs has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................


Patch Set 9:

(5 comments)

A few comments.

https://asterix-gerrit.ics.uci.edu/#/c/427/9/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionVisitor.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionVisitor.java:

Line 66:  */
Wrong author here?


https://asterix-gerrit.ics.uci.edu/#/c/427/9/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java:

Line 86:         if (-1 != partition)
Just a stylistic thing here: I think this is better as 
"partition != -1"


https://asterix-gerrit.ics.uci.edu/#/c/427/9/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java:

Line 80:         if (-1 != partition)
Same here


https://asterix-gerrit.ics.uci.edu/#/c/427/9/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/OrderedPartitionedProperty.java:

Line 92:     
whitespace


https://asterix-gerrit.ics.uci.edu/#/c/427/9/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
File algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java:

Line 292:             if (null != rqd.getPartitioningProperty() && rqd.getPartitioningProperty().isSingle()) {
Style again, should be "rgd... != null"


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 9
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: Yes

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................


Patch Set 10:

(26 comments)

Here are some high-level comments:

1. We need to revisit what needs to be added into the partitioning property to indicate where the operator actually needs to be run.  I'm not sure the abstraction of "isSingle" is enough. I inlined two motivating queries.

2. The caller of an IMetadataProvider probably shouldn't be a aware of a node id, but a hash value, or more general, the INodeDomain information. I think this also needs a discussion.

3. The code style seems not follow the recommended one: 
http://asterixdb.incubator.apache.org/dev-setup.html

Other detailed comments are inlined.

https://asterix-gerrit.ics.uci.edu/#/c/427/10/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java:

Line 71:             JobGenContext context, JobSpecification jobSpec, boolean bulkload, int nodeId) throws AlgebricksException;
int nodeId->Integer hashValue

I guess the caller shouldn't be aware of nodeId, but only hashValue.
if the Integer is null, that means the reader needs to run on all storage nodes.


Line 75:             LogicalVariable payLoadVar, List<LogicalVariable> additionalNonKeyFields, RecordDescriptor recordDesc,
int nodeId->Integer hashValue

I guess the caller shouldn't be aware of nodeId, but only hashValue.
if the Integer is null, that means the reader needs to run on all storage nodes.


Line 115:             JobGenContext context, JobSpecification spec, boolean bulkload, int nodeId) throws AlgebricksException;
int nodeId->Integer hashValue

I guess the caller shouldn't be aware of nodeId, but only hashValue.
if the Integer is null, that means the reader needs to run on all storage nodes.


Line 154:             JobGenContext context, JobSpecification spec, int nodeId) throws AlgebricksException;
int nodeId->Integer hashValue

I guess the caller shouldn't be aware of nodeId, but only hashValue.
if the Integer is null, that means the reader needs to run on all storage nodes.


Line 199:             RecordDescriptor recordDesc, JobGenContext context, JobSpecification jobSpec, int nodeId)
int nodeId->Integer hashValue

I guess the caller shouldn't be aware of nodeId, but only hashValue.
if the Integer is null, that means the reader needs to run on all storage nodes.


Line 207:             RecordDescriptor inputDesc, JobGenContext context, JobSpecification spec, int nodeId)
int nodeId->Integer hashValue

I guess the caller shouldn't be aware of nodeId, but only hashValue.
if the Integer is null, that means the reader needs to run on all storage nodes.


https://asterix-gerrit.ics.uci.edu/#/c/427/10/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java:

Line 316:                     + GroupByOperator.veListToString(gByList) + " to " + GroupByOperator.veListToString(newGbyList)
It seems your code style is not right.


https://asterix-gerrit.ics.uci.edu/#/c/427/10/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionVisitor.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionVisitor.java:

Line 66:  * @author whli
add some comments to describe what this class is used for.


Line 72:     public ProducedVariableExpressionVisitor(Map<List<LogicalVariable>, List<ILogicalExpression>> varToExpr) {
Why not map a variable to an expression?


Line 78:         return null;
aggregate produces variables.


Line 83:         return null;
running aggregate produces variables.


Line 93:         return null;
group by produces variables.


Line 129:         varToExpr.put(vars, exprs);
why not map each individual variable to its definition expression?


Line 169:     public Void visitSubplanOperator(SubplanOperator op, Void arg) throws AlgebricksException {
subplan produces variables.


Line 180:         // TODO Auto-generated method stub
union produces variables.


Line 186:         // TODO Auto-generated method stub
unnest produces variables.


Line 204:     }
data scan produces variables.


Line 234:         List<LogicalVariable> vars = new ArrayList<LogicalVariable>();
insertdelete doesn't produces any variables.
Here the code maps used variables to null?


Line 256:         // TODO Auto-generated method stub
intersect produces variables.


Line 262:         // TODO Auto-generated method stub
outer unnest produces variables.


https://asterix-gerrit.ics.uci.edu/#/c/427/10/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java:

Line 106:                 inputDesc, context, spec, true, -1);
pass in null instead of -1.


https://asterix-gerrit.ics.uci.edu/#/c/427/10/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java:

Line 131:                 additionalFilteringKeys, filterExpr, inputDesc, context, spec, true, -1);
-1 -> null


https://asterix-gerrit.ics.uci.edu/#/c/427/10/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java:

Line 105:             ((UnorderedPartitionedProperty) r.getPartitioningProperty()).setSingle(this.isSinglePartition());
There shouldn't be a special handling of isSinglePart.


https://asterix-gerrit.ics.uci.edu/#/c/427/10/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/IPartitioningProperty.java:

Line 94:     public void setSingle(boolean isSingle);
This looks to be an immutable field, thus it's better to remove the setter and enforce that in the constructor.


Line 96:     public boolean isSingle();
I'm not sure whether isSingle() is enough to capture the partitioning property.

You also need to know which partition(s) are involved.

for example, consider the following query:

create dataset foo(id int32, value string);
create dataset bar(id int32, value string);

insert into dataset foo(
  for $d in dataset bar
  where $d.key = 1
  return {
     "key":  $d.key + 3,
     "value":  $d.value
  }
);

The primary key search for bar goes to one node and the insert to foo goes to one node.  Both datasets are hash partitioned in the same way. But the two nodes could be different nodes.

I'm thinking where we can extend DefaultNodeGroupDomain with a boolean vector to indicate where the operator instances need to be run.

Also consider the following query (more general cases):

for $d in dataset bar
where $d.key = 1 OR $d.key=2
return $d

It would be nice that the query can only go to two nodes  (or one node), depending on the hash function.  That also seems to require an more general abstraction than isSingle.


https://asterix-gerrit.ics.uci.edu/#/c/427/10/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
File algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java:

Line 97: 
it seems the code style is not right.


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 10
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: Yes

Change in hyracks[master]: Exchange optimization for lookup and delete

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Wenhai Li has uploaded a new patch set (#2).

Change subject: Exchange optimization for lookup and delete
......................................................................

Exchange optimization for lookup and delete

The following commits from your working branch will be included:
exchange optimization for lookup and delete when the operation is merely directed by the primary key.

commit 8d803cab358382b7c1510900cc7551de5fcc8480
Author: Michael <lw...@yahoo.com>
Date:   Thu Oct 1 23:58:48 2015 -0700

    exchange optimization for lookup and delete

Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
M algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
3 files changed, 130 insertions(+), 14 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/27/427/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 2
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Wenhai Li (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/427

to look at the new patch set (#7).

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................

Equivalence induced onetoone operatiions for insert/delete/upsert/query
exchange optimization for lookup and delete when the operation is merely directed by the primary key.

commit 8d803cab358382b7c1510900cc7551de5fcc8480
Author: Michael <lw...@yahoo.com>
Date:   Thu Oct 1 23:58:48 2015 -0700

    exchange optimization for lookup and delete

Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.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/RandomPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
M algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
16 files changed, 421 insertions(+), 38 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/27/427/7
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 7
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................


Patch Set 10: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/912/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 10
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Steven Jacobs (Code Review)" <do...@asterixdb.incubator.apache.org>.
Steven Jacobs has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................


Patch Set 9:

(1 comment)

One bonus comment after looking at Asterix. It seems like IsSinglePartition isn't actually needed
Steven

https://asterix-gerrit.ics.uci.edu/#/c/427/9/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java:

Line 61: 
After looking at Asterix, I think the same comment applies here. Do we actually need "isSinglePartition" or can we just check by "partitionId == -1"


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 9
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: Yes

Change in hyracks[master]: Equivalence induced onetoone operatiions

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/536/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/883/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 6
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................


Patch Set 9: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/888/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 9
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Exchange optimization for lookup and delete

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Exchange optimization for lookup and delete
......................................................................


Patch Set 1: Verified-1

Build Failed 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/482/ : ABORTED

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 1
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Ian Maxon (Code Review)" <do...@asterixdb.incubator.apache.org>.
Ian Maxon has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query
......................................................................


Patch Set 14: Verified-1

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 14
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Exchange optimization for lookup and delete

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Exchange optimization for lookup and delete
......................................................................


Patch Set 2: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/483/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 2
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query
......................................................................


Patch Set 14: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/973/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 14
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Steven Jacobs, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/427

to look at the new patch set (#14).

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query
......................................................................

Equivalence induced onetoone operatiions for insert/delete/upsert/query

commit 8d803cab358382b7c1510900cc7551de5fcc8480
Author: Michael <lw...@yahoo.com>
Date:   Thu Oct 1 23:58:48 2015 -0700

    exchange optimization for lookup and delete

Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteUpsertOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionListVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/DefaultNodeGroupDomain.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/FileSplitDomain.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/INodeDomain.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/RandomPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/ResultSetDomain.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
M algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
21 files changed, 512 insertions(+), 87 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/27/427/14
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 14
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in hyracks[master]: Equivalence induced onetoone operatiions

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions
......................................................................


Patch Set 5: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/537/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 5
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Wenhai Li (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/427

to look at the new patch set (#8).

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................

Equivalence induced onetoone operatiions for insert/delete/upsert/query
exchange optimization for lookup and delete when the operation is merely directed by the primary key.

commit 8d803cab358382b7c1510900cc7551de5fcc8480
Author: Michael <lw...@yahoo.com>
Date:   Thu Oct 1 23:58:48 2015 -0700

    exchange optimization for lookup and delete

Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.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/RandomPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
M algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
16 files changed, 421 insertions(+), 38 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/27/427/8
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 8
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in hyracks[master]: Equivalence induced onetoone operatiions

Posted by "Wenhai Li (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/427

to look at the new patch set (#5).

Change subject: Equivalence induced onetoone operatiions
......................................................................

Equivalence induced onetoone operatiions

exchange optimization for lookup and delete when the operation is merely directed by the primary key.

commit 8d803cab358382b7c1510900cc7551de5fcc8480
Author: Michael <lw...@yahoo.com>
Date:   Thu Oct 1 23:58:48 2015 -0700

    exchange optimization for lookup and delete

Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.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/RandomPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
M algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
15 files changed, 421 insertions(+), 19 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/27/427/5
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 5
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query exchange optimization for lookup and delete when the operation is merely directed by the primary key.
......................................................................


Patch Set 7: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/884/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 7
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query
......................................................................


Patch Set 13: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/967/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 13
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query
......................................................................


Patch Set 14:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/973/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 14
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query
......................................................................


Patch Set 11: Verified+1

Build Successful 

https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/956/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 11
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Steven Jacobs, Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/427

to look at the new patch set (#12).

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query
......................................................................

Equivalence induced onetoone operatiions for insert/delete/upsert/query

commit 8d803cab358382b7c1510900cc7551de5fcc8480
Author: Michael <lw...@yahoo.com>
Date:   Thu Oct 1 23:58:48 2015 -0700

    exchange optimization for lookup and delete

Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionListVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeleteUpsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.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/RandomPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
M algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
16 files changed, 497 insertions(+), 80 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/27/427/12
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 12
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in hyracks[master]: PLEASE EDIT to provide a meaningful commit message!

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: PLEASE EDIT to provide a meaningful commit message!
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/482/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 1
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: Equivalence induced onetoone operatiions

Posted by "Wenhai Li (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/427

to look at the new patch set (#4).

Change subject: Equivalence induced onetoone operatiions
......................................................................

Equivalence induced onetoone operatiions

The following commits from your working branch will be included:
exchange optimization for lookup and delete when the operation is merely directed by the primary key.

commit 8d803cab358382b7c1510900cc7551de5fcc8480
Author: Michael <lw...@yahoo.com>
Date:   Thu Oct 1 23:58:48 2015 -0700

    exchange optimization for lookup and delete

Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/UnnestMapOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableExpressionVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/BroadcastPartitioningProperty.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/RandomPartitioningProperty.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/UnorderedPartitionedProperty.java
M algebricks/algebricks-examples/piglet-example/src/main/java/org/apache/hyracks/algebricks/examples/piglet/metadata/PigletMetadataProvider.java
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
15 files changed, 421 insertions(+), 19 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/hyracks refs/changes/27/427/4
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in hyracks[master]: Equivalence induced onetoone operatiions for insert/delete/u...

Posted by "Wenhai Li (Code Review)" <do...@asterixdb.incubator.apache.org>.
Wenhai Li has abandoned this change.

Change subject: Equivalence induced onetoone operatiions for insert/delete/upsert/query
......................................................................


Abandoned

Merge onto the new patch https://asterix-gerrit.ics.uci.edu/#/c/428/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/427
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: abandon
Gerrit-Change-Id: Ic836210e57b87128120e1f8dabbfed062d09f5e4
Gerrit-PatchSet: 14
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Jianfeng Jia <ji...@gmail.com>
Gerrit-Reviewer: Steven Jacobs <sj...@ucr.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Wenhai Li <lw...@yahoo.com>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>