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

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

Ildar Absalyamov has uploaded a new change for review.

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................

ASTERIXDB-1109: Fixed deletion of records from open secondary index

- Separated InsertDeleteOperator into two distinct operators
- Added a rule, which reuses PK vars, retrieved from datascan\idxLookup
- Changed plan printer for insert\delete\idxInsert\idxDelete operators
- Modified plan printer to skip repeated parts of the plan under replicate operator

Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractModificationOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DeleteOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/TokenizeOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
C algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DeletePOperator.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/IndexInsertDeletePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/TokenizePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/PlanPrettyPrinter.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.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
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
M algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
32 files changed, 569 insertions(+), 369 deletions(-)


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

diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
index 6668536..a7b5908 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
@@ -31,7 +31,8 @@
     GROUP,
     INDEX_INSERT_DELETE,
     INNERJOIN,
-    INSERT_DELETE,
+    INSERT,
+    DELETE,
     LEFTOUTERJOIN,
     LIMIT,
     MATERIALIZE,
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
index c581e82..8b01e2f 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -39,7 +39,8 @@
     IN_MEMORY_STABLE_SORT,
     INDEX_BULKLOAD,
     INDEX_INSERT_DELETE,
-    INSERT_DELETE,
+    INSERT,
+    DELETE,
     LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH,
     MATERIALIZE,
     MICRO_PRE_CLUSTERED_GROUP_BY,
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java
index bab23f1..ea5eef1 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java
@@ -27,7 +27,6 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
-
 import org.apache.hyracks.algebricks.core.algebra.base.EquivalenceClass;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
@@ -124,6 +123,18 @@
             sb.append(ref.getValue());
         }
         sb.append("]");
+        if (opaqueParameters != null) {
+            sb.append(", OpaqueArgs:[");
+            first = true;
+            for (Object param : opaqueParameters) {
+                if (first) {
+                    first = false;
+                } else {
+                    sb.append(", ");
+                }
+                sb.append(param);
+            }
+        }
         return sb.toString();
     }
 
@@ -172,7 +183,8 @@
     }
 
     @Override
-    public void getConstraintsForOuterJoin(Collection<FunctionalDependency> fds, Collection<LogicalVariable> outerVars) {
+    public void getConstraintsForOuterJoin(Collection<FunctionalDependency> fds,
+            Collection<LogicalVariable> outerVars) {
         FunctionIdentifier funId = getFunctionIdentifier();
         if (funId.equals(AlgebricksBuiltinFunctions.AND)) {
             for (Mutable<ILogicalExpression> a : arguments) {
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..03f20ea 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
@@ -48,13 +48,13 @@
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
             List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
-            throws AlgebricksException;
+                    throws AlgebricksException;
 
     public boolean scannerOperatorIsLeaf(IDataSource<S> dataSource);
 
     public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteFileRuntime(IDataSink sink,
             int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc)
-            throws AlgebricksException;
+                    throws AlgebricksException;
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getResultHandleRuntime(IDataSink sink,
             int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc, boolean ordered,
@@ -63,7 +63,7 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getWriteResultRuntime(IDataSource<S> dataSource,
             IOperatorSchema propagatedSchema, List<LogicalVariable> keys, LogicalVariable payLoadVar,
             List<LogicalVariable> additionalNonKeyFields, JobGenContext context, JobSpecification jobSpec)
-            throws AlgebricksException;
+                    throws AlgebricksException;
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(IDataSource<S> dataSource,
             IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv, List<LogicalVariable> keys,
@@ -72,8 +72,8 @@
 
     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) throws AlgebricksException;
+            List<LogicalVariable> additionalNonKeyFields, RecordDescriptor recordDesc, JobGenContext context,
+            JobSpecification jobSpec) throws AlgebricksException;
 
     /**
      * Creates the insert runtime of IndexInsertDeletePOperator, which models
@@ -191,6 +191,8 @@
 
     public IDataSourceIndex<I, S> findDataSourceIndex(I indexId, S dataSourceId) throws AlgebricksException;
 
+    public IDataSourceIndex<I, S> findIndex(I indexId, I dataverseName, I datasetName) throws AlgebricksException;
+
     public IFunctionInfo lookupFunction(FunctionIdentifier fid);
 
 }
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractModificationOperator.java
similarity index 68%
rename from algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteOperator.java
rename to algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractModificationOperator.java
index b95d279..8d845cc 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertDeleteOperator.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractModificationOperator.java
@@ -22,43 +22,26 @@
 import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-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.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
 import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
-import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 
-public class InsertDeleteOperator extends AbstractLogicalOperator {
+public abstract class AbstractModificationOperator extends AbstractLogicalOperator {
 
-    public enum Kind {
-        INSERT,
-        DELETE
-    }
+    protected final IDataSource<?> dataSource;
 
-    private final IDataSource<?> dataSource;
+    protected final List<Mutable<ILogicalExpression>> primaryKeyExprs;
 
-    private final Mutable<ILogicalExpression> payloadExpr;
-
-    private final List<Mutable<ILogicalExpression>> primaryKeyExprs;
-
-    private final Kind operation;
     private List<Mutable<ILogicalExpression>> additionalFilteringExpressions;
 
-    private final boolean bulkload;
-
-    public InsertDeleteOperator(IDataSource<?> dataSource, Mutable<ILogicalExpression> payloadExpr,
-            List<Mutable<ILogicalExpression>> primaryKeyExprs, Kind operation, boolean bulkload) {
+    public AbstractModificationOperator(IDataSource<?> dataSource, List<Mutable<ILogicalExpression>> primaryKeyExprs) {
         this.dataSource = dataSource;
-        this.payloadExpr = payloadExpr;
         this.primaryKeyExprs = primaryKeyExprs;
-        this.operation = operation;
-        this.bulkload = bulkload;
     }
 
     @Override
@@ -68,18 +51,13 @@
     }
 
     @Override
-    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform transform) throws AlgebricksException {
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform transform)
+            throws AlgebricksException {
         boolean changed = false;
-        changed = transform.transform(payloadExpr);
         for (Mutable<ILogicalExpression> e : primaryKeyExprs) {
             changed |= transform.transform(e);
         }
         return changed;
-    }
-
-    @Override
-    public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
-        return visitor.visitInsertDeleteOperator(this, arg);
     }
 
     @Override
@@ -90,11 +68,6 @@
     @Override
     public VariablePropagationPolicy getVariablePropagationPolicy() {
         return VariablePropagationPolicy.ALL;
-    }
-
-    @Override
-    public LogicalOperatorTag getOperatorTag() {
-        return LogicalOperatorTag.INSERT_DELETE;
     }
 
     @Override
@@ -109,18 +82,6 @@
     public IDataSource<?> getDataSource() {
         return dataSource;
     }
-
-    public Mutable<ILogicalExpression> getPayloadExpression() {
-        return payloadExpr;
-    }
-
-    public Kind getOperation() {
-        return operation;
-    }
-
-    public boolean isBulkload() {
-        return bulkload;
-	}
 
     public void setAdditionalFilteringExpressions(List<Mutable<ILogicalExpression>> additionalFilteringExpressions) {
         this.additionalFilteringExpressions = additionalFilteringExpressions;
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DeleteOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DeleteOperator.java
new file mode 100644
index 0000000..90fb45e
--- /dev/null
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DeleteOperator.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+
+public class DeleteOperator extends AbstractModificationOperator {
+
+    public DeleteOperator(IDataSource<?> dataSource, List<Mutable<ILogicalExpression>> primaryKeyExprs) {
+        super(dataSource, primaryKeyExprs);
+    }
+
+    @Override
+    public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+        return visitor.visitDeleteOperator(this, arg);
+    }
+
+    @Override
+    public LogicalOperatorTag getOperatorTag() {
+        return LogicalOperatorTag.DELETE;
+    }
+
+}
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteOperator.java
index e9e3b01..0013669 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteOperator.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteOperator.java
@@ -22,15 +22,12 @@
 import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 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.expressions.IVariableTypeEnvironment;
-import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator.Kind;
 import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
@@ -38,6 +35,11 @@
 
 public class IndexInsertDeleteOperator extends AbstractLogicalOperator {
 
+    public enum Kind {
+        INSERT,
+        DELETE
+    }
+
     private final IDataSourceIndex<?, ?> dataSourceIndex;
     private final List<Mutable<ILogicalExpression>> primaryKeyExprs;
     // In the bulk-load case on ngram or keyword index,
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertOperator.java
new file mode 100644
index 0000000..2926535
--- /dev/null
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertOperator.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.hyracks.algebricks.core.algebra.operators.logical;
+
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSource;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+
+public class InsertOperator extends AbstractModificationOperator {
+
+    private final Mutable<ILogicalExpression> payloadExpr;
+
+    private final boolean bulkload;
+
+    public InsertOperator(IDataSource<?> dataSource, Mutable<ILogicalExpression> payloadExpr,
+            List<Mutable<ILogicalExpression>> primaryKeyExprs, boolean bulkload) {
+        super(dataSource, primaryKeyExprs);
+        this.payloadExpr = payloadExpr;
+        this.bulkload = bulkload;
+    }
+
+    @Override
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform transform)
+            throws AlgebricksException {
+        boolean changed = false;
+        changed = transform.transform(payloadExpr);
+        for (Mutable<ILogicalExpression> e : primaryKeyExprs) {
+            changed |= transform.transform(e);
+        }
+        return changed;
+    }
+
+    @Override
+    public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+        return visitor.visitInsertOperator(this, arg);
+    }
+
+    @Override
+    public LogicalOperatorTag getOperatorTag() {
+        return LogicalOperatorTag.INSERT;
+    }
+
+    public Mutable<ILogicalExpression> getPayloadExpression() {
+        return payloadExpr;
+    }
+
+    public boolean isBulkload() {
+        return bulkload;
+    }
+
+}
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/TokenizeOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/TokenizeOperator.java
index e1a793c..081ed3a 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/TokenizeOperator.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/TokenizeOperator.java
@@ -22,14 +22,13 @@
 import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 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.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator.Kind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator.Kind;
 import org.apache.hyracks.algebricks.core.algebra.properties.VariablePropagationPolicy;
 import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
@@ -50,12 +49,9 @@
     private final List<Object> tokenizeVarTypes;
     private List<Mutable<ILogicalExpression>> additionalFilteringExpressions;
 
-    public TokenizeOperator(IDataSourceIndex<?, ?> dataSourceIndex,
-            List<Mutable<ILogicalExpression>> primaryKeyExprs,
-            List<Mutable<ILogicalExpression>> secondaryKeyExprs,
-            List<LogicalVariable> tokenizeVars,
-            Mutable<ILogicalExpression> filterExpr, Kind operation,
-            boolean bulkload, boolean isPartitioned,
+    public TokenizeOperator(IDataSourceIndex<?, ?> dataSourceIndex, List<Mutable<ILogicalExpression>> primaryKeyExprs,
+            List<Mutable<ILogicalExpression>> secondaryKeyExprs, List<LogicalVariable> tokenizeVars,
+            Mutable<ILogicalExpression> filterExpr, Kind operation, boolean bulkload, boolean isPartitioned,
             List<Object> tokenizeVarTypes) {
         this.dataSourceIndex = dataSourceIndex;
         this.primaryKeyExprs = primaryKeyExprs;
@@ -76,9 +72,7 @@
     }
 
     @Override
-    public boolean acceptExpressionTransform(
-            ILogicalExpressionReferenceTransform visitor)
-            throws AlgebricksException {
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
         boolean b = false;
         for (int i = 0; i < primaryKeyExprs.size(); i++) {
             if (visitor.transform(primaryKeyExprs.get(i))) {
@@ -94,8 +88,7 @@
     }
 
     @Override
-    public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg)
-            throws AlgebricksException {
+    public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
         return visitor.visitTokenizeOperator(this, arg);
     }
 
@@ -109,8 +102,8 @@
         return new VariablePropagationPolicy() {
 
             @Override
-            public void propagateVariables(IOperatorSchema target,
-                    IOperatorSchema... sources) throws AlgebricksException {
+            public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
+                    throws AlgebricksException {
                 target.addAllVariables(sources[0]);
                 for (LogicalVariable v : tokenizeVars) {
                     target.addVariable(v);
@@ -126,8 +119,7 @@
     }
 
     @Override
-    public IVariableTypeEnvironment computeOutputTypeEnvironment(
-            ITypingContext ctx) throws AlgebricksException {
+    public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
         IVariableTypeEnvironment env = createPropagatingAllInputsTypeEnvironment(ctx);
 
         // If the secondary index is not length-partitioned, create one new
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
index 6d00fab..183890c 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
@@ -29,7 +29,6 @@
 import java.util.Set;
 
 import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -50,6 +49,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
@@ -59,7 +59,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
@@ -311,9 +311,9 @@
             }
         }
         if (changed) {
-            AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Group-by list changed from "
-                    + GroupByOperator.veListToString(gByList) + " to " + GroupByOperator.veListToString(newGbyList)
-                    + ".\n");
+            AlgebricksConfig.ALGEBRICKS_LOGGER
+                    .fine(">>>> Group-by list changed from " + GroupByOperator.veListToString(gByList) + " to "
+                            + GroupByOperator.veListToString(newGbyList) + ".\n");
         }
         gByList.clear();
         gByList.addAll(newGbyList);
@@ -509,7 +509,13 @@
     }
 
     @Override
-    public Void visitInsertDeleteOperator(InsertDeleteOperator op, IOptimizationContext ctx) throws AlgebricksException {
+    public Void visitInsertOperator(InsertOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        propagateFDsAndEquivClasses(op, ctx);
+        return null;
+    }
+
+    @Override
+    public Void visitDeleteOperator(DeleteOperator op, IOptimizationContext ctx) throws AlgebricksException {
         propagateFDsAndEquivClasses(op, ctx);
         return null;
     }
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index a982ed4..0cf6cdb 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -25,7 +25,6 @@
 import java.util.Map.Entry;
 
 import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
@@ -38,6 +37,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
@@ -47,7 +47,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
@@ -453,11 +453,11 @@
     }
 
     @Override
-    public Boolean visitInsertDeleteOperator(InsertDeleteOperator op, ILogicalOperator arg) throws AlgebricksException {
+    public Boolean visitInsertOperator(InsertOperator op, ILogicalOperator arg) throws AlgebricksException {
         AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
-        if (aop.getOperatorTag() != LogicalOperatorTag.INSERT_DELETE)
+        if (aop.getOperatorTag() != LogicalOperatorTag.INSERT)
             return Boolean.FALSE;
-        InsertDeleteOperator insertOpArg = (InsertDeleteOperator) copyAndSubstituteVar(op, arg);
+        InsertOperator insertOpArg = (InsertOperator) copyAndSubstituteVar(op, arg);
         boolean isomorphic = VariableUtilities.varListEqualUnordered(op.getSchema(), insertOpArg.getSchema());
         if (!op.getDataSource().equals(insertOpArg.getDataSource()))
             isomorphic = false;
@@ -467,6 +467,18 @@
     }
 
     @Override
+    public Boolean visitDeleteOperator(DeleteOperator op, ILogicalOperator arg) throws AlgebricksException {
+        AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
+        if (aop.getOperatorTag() != LogicalOperatorTag.DELETE)
+            return Boolean.FALSE;
+        DeleteOperator deleteOpArg = (DeleteOperator) copyAndSubstituteVar(op, arg);
+        boolean isomorphic = VariableUtilities.varListEqualUnordered(op.getSchema(), deleteOpArg.getSchema());
+        if (!op.getDataSource().equals(deleteOpArg.getDataSource()))
+            isomorphic = false;
+        return isomorphic;
+    }
+
+    @Override
     public Boolean visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, ILogicalOperator arg)
             throws AlgebricksException {
         AbstractLogicalOperator aop = (AbstractLogicalOperator) arg;
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
index 4f0bcd3..694d4d8 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
@@ -25,7 +25,6 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
@@ -39,6 +38,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
@@ -48,7 +48,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
@@ -250,7 +250,13 @@
     }
 
     @Override
-    public Void visitInsertDeleteOperator(InsertDeleteOperator op, ILogicalOperator arg) throws AlgebricksException {
+    public Void visitInsertOperator(InsertOperator op, ILogicalOperator arg) throws AlgebricksException {
+        mapVariablesStandard(op, arg);
+        return null;
+    }
+
+    @Override
+    public Void visitDeleteOperator(DeleteOperator op, ILogicalOperator arg) throws AlgebricksException {
         mapVariablesStandard(op, arg);
         return null;
     }
@@ -263,8 +269,7 @@
     }
 
     @Override
-    public Void visitTokenizeOperator(TokenizeOperator op, ILogicalOperator arg)
-            throws AlgebricksException {
+    public Void visitTokenizeOperator(TokenizeOperator op, ILogicalOperator arg) throws AlgebricksException {
         mapVariablesStandard(op, arg);
         return null;
     }
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
index 32dbc94..0283e05 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
@@ -19,7 +19,6 @@
 package org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors;
 
 import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
@@ -31,6 +30,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
@@ -40,7 +40,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
@@ -80,8 +80,8 @@
         }
         op.accept(visitor, context);
         if (AlgebricksConfig.DEBUG) {
-            AlgebricksConfig.ALGEBRICKS_LOGGER.finest("Logical properties visitor for " + op + ": "
-                    + context.getLogicalPropertiesVector(op) + "\n");
+            AlgebricksConfig.ALGEBRICKS_LOGGER.finest(
+                    "Logical properties visitor for " + op + ": " + context.getLogicalPropertiesVector(op) + "\n");
         }
     }
 
@@ -259,7 +259,13 @@
     }
 
     @Override
-    public Void visitInsertDeleteOperator(InsertDeleteOperator op, IOptimizationContext arg) throws AlgebricksException {
+    public Void visitInsertOperator(InsertOperator op, IOptimizationContext arg) throws AlgebricksException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public Void visitDeleteOperator(DeleteOperator op, IOptimizationContext arg) throws AlgebricksException {
         // TODO Auto-generated method stub
         return null;
     }
@@ -272,8 +278,7 @@
     }
 
     @Override
-    public Void visitTokenizeOperator(TokenizeOperator op, IOptimizationContext arg)
-            throws AlgebricksException {
+    public Void visitTokenizeOperator(TokenizeOperator op, IOptimizationContext arg) throws AlgebricksException {
         // TODO Auto-generated method stub
         return null;
     }
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
index afc832a..439f5c7 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
@@ -24,7 +24,6 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
@@ -36,6 +35,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
@@ -45,7 +45,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
@@ -115,20 +115,20 @@
 
     @Override
     public ILogicalOperator visitLimitOperator(LimitOperator op, Void arg) throws AlgebricksException {
-        return new LimitOperator(deepCopyExpressionRef(op.getMaxObjects()).getValue(), deepCopyExpressionRef(
-                op.getOffset()).getValue(), op.isTopmostLimitOp());
+        return new LimitOperator(deepCopyExpressionRef(op.getMaxObjects()).getValue(),
+                deepCopyExpressionRef(op.getOffset()).getValue(), op.isTopmostLimitOp());
     }
 
     @Override
     public ILogicalOperator visitInnerJoinOperator(InnerJoinOperator op, Void arg) throws AlgebricksException {
-        return new InnerJoinOperator(deepCopyExpressionRef(op.getCondition()), op.getInputs().get(0), op.getInputs()
-                .get(1));
+        return new InnerJoinOperator(deepCopyExpressionRef(op.getCondition()), op.getInputs().get(0),
+                op.getInputs().get(1));
     }
 
     @Override
     public ILogicalOperator visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void arg) throws AlgebricksException {
-        return new LeftOuterJoinOperator(deepCopyExpressionRef(op.getCondition()), op.getInputs().get(0), op
-                .getInputs().get(1));
+        return new LeftOuterJoinOperator(deepCopyExpressionRef(op.getCondition()), op.getInputs().get(0),
+                op.getInputs().get(1));
     }
 
     @Override
@@ -215,8 +215,8 @@
     public ILogicalOperator visitUnnestMapOperator(UnnestMapOperator op, Void arg) throws AlgebricksException {
         ArrayList<LogicalVariable> newInputList = new ArrayList<LogicalVariable>();
         newInputList.addAll(op.getVariables());
-        return new UnnestMapOperator(newInputList, deepCopyExpressionRef(op.getExpressionRef()), new ArrayList<Object>(
-                op.getVariableTypes()), op.propagatesInput());
+        return new UnnestMapOperator(newInputList, deepCopyExpressionRef(op.getExpressionRef()),
+                new ArrayList<Object>(op.getVariableTypes()), op.propagatesInput());
     }
 
     @Override
@@ -266,15 +266,26 @@
     }
 
     @Override
-    public ILogicalOperator visitInsertDeleteOperator(InsertDeleteOperator op, Void arg) throws AlgebricksException {
+    public ILogicalOperator visitInsertOperator(InsertOperator op, Void arg) throws AlgebricksException {
         List<Mutable<ILogicalExpression>> newKeyExpressions = new ArrayList<Mutable<ILogicalExpression>>();
         deepCopyExpressionRefs(newKeyExpressions, op.getPrimaryKeyExpressions());
         List<Mutable<ILogicalExpression>> newLSMComponentFilterExpressions = new ArrayList<Mutable<ILogicalExpression>>();
         deepCopyExpressionRefs(newKeyExpressions, op.getAdditionalFilteringExpressions());
-        InsertDeleteOperator insertDeleteOp = new InsertDeleteOperator(op.getDataSource(),
-                deepCopyExpressionRef(op.getPayloadExpression()), newKeyExpressions, op.getOperation(), op.isBulkload());
-        insertDeleteOp.setAdditionalFilteringExpressions(newLSMComponentFilterExpressions);
-        return insertDeleteOp;
+        InsertOperator insertOp = new InsertOperator(op.getDataSource(),
+                deepCopyExpressionRef(op.getPayloadExpression()), newKeyExpressions, op.isBulkload());
+        insertOp.setAdditionalFilteringExpressions(newLSMComponentFilterExpressions);
+        return insertOp;
+    }
+
+    @Override
+    public ILogicalOperator visitDeleteOperator(DeleteOperator op, Void arg) throws AlgebricksException {
+        List<Mutable<ILogicalExpression>> newKeyExpressions = new ArrayList<Mutable<ILogicalExpression>>();
+        deepCopyExpressionRefs(newKeyExpressions, op.getPrimaryKeyExpressions());
+        List<Mutable<ILogicalExpression>> newLSMComponentFilterExpressions = new ArrayList<Mutable<ILogicalExpression>>();
+        deepCopyExpressionRefs(newKeyExpressions, op.getAdditionalFilteringExpressions());
+        DeleteOperator deleteOp = new DeleteOperator(op.getDataSource(), newKeyExpressions);
+        deleteOp.setAdditionalFilteringExpressions(newLSMComponentFilterExpressions);
+        return deleteOp;
     }
 
     @Override
@@ -322,12 +333,13 @@
     private void deepCopyExpressionRefs(List<Mutable<ILogicalExpression>> newExprs,
             List<Mutable<ILogicalExpression>> oldExprs) {
         for (Mutable<ILogicalExpression> oldExpr : oldExprs)
-            newExprs.add(new MutableObject<ILogicalExpression>(((AbstractLogicalExpression) oldExpr.getValue())
-                    .cloneExpression()));
+            newExprs.add(new MutableObject<ILogicalExpression>(
+                    ((AbstractLogicalExpression) oldExpr.getValue()).cloneExpression()));
     }
 
     private Mutable<ILogicalExpression> deepCopyExpressionRef(Mutable<ILogicalExpression> oldExpr) {
-        return new MutableObject<ILogicalExpression>(((AbstractLogicalExpression) oldExpr.getValue()).cloneExpression());
+        return new MutableObject<ILogicalExpression>(
+                ((AbstractLogicalExpression) oldExpr.getValue()).cloneExpression());
     }
 
     private List<LogicalVariable> deepCopyVars(List<LogicalVariable> newVars, List<LogicalVariable> oldVars) {
@@ -346,8 +358,8 @@
             List<Pair<IOrder, Mutable<ILogicalExpression>>> ordersAndExprs) {
         List<Pair<IOrder, Mutable<ILogicalExpression>>> newOrdersAndExprs = new ArrayList<Pair<IOrder, Mutable<ILogicalExpression>>>();
         for (Pair<IOrder, Mutable<ILogicalExpression>> pair : ordersAndExprs)
-            newOrdersAndExprs.add(new Pair<IOrder, Mutable<ILogicalExpression>>(pair.first,
-                    deepCopyExpressionRef(pair.second)));
+            newOrdersAndExprs
+                    .add(new Pair<IOrder, Mutable<ILogicalExpression>>(pair.first, deepCopyExpressionRef(pair.second)));
         return newOrdersAndExprs;
     }
 
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
index 8383ef2..c3816d8 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
@@ -23,7 +23,6 @@
 import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
@@ -34,6 +33,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
@@ -43,7 +43,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
@@ -245,7 +245,12 @@
     }
 
     @Override
-    public Void visitInsertDeleteOperator(InsertDeleteOperator op, Void arg) throws AlgebricksException {
+    public Void visitInsertOperator(InsertOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
+    public Void visitDeleteOperator(DeleteOperator op, Void arg) throws AlgebricksException {
         return null;
     }
 
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
index 29b7edd..52954fa 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
@@ -23,7 +23,6 @@
 import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -35,6 +34,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
@@ -44,7 +44,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
@@ -282,7 +282,13 @@
     }
 
     @Override
-    public Void visitInsertDeleteOperator(InsertDeleteOperator op, Void arg) throws AlgebricksException {
+    public Void visitInsertOperator(InsertOperator op, Void arg) throws AlgebricksException {
+        standardLayout(op);
+        return null;
+    }
+
+    @Override
+    public Void visitDeleteOperator(DeleteOperator op, Void arg) throws AlgebricksException {
         standardLayout(op);
         return null;
     }
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
index 77e6cb6..d858e7f 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -22,7 +22,6 @@
 import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
@@ -35,6 +34,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
@@ -44,7 +44,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
@@ -70,7 +70,8 @@
 import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 
-public class SubstituteVariableVisitor implements ILogicalOperatorVisitor<Void, Pair<LogicalVariable, LogicalVariable>> {
+public class SubstituteVariableVisitor
+        implements ILogicalOperatorVisitor<Void, Pair<LogicalVariable, LogicalVariable>> {
 
     private final boolean goThroughNts;
     private final ITypingContext ctx;
@@ -147,7 +148,8 @@
     }
 
     @Override
-    public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Pair<LogicalVariable, LogicalVariable> pair) {
+    public Void visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op,
+            Pair<LogicalVariable, LogicalVariable> pair) {
         // does not use any variable
         return null;
     }
@@ -202,8 +204,8 @@
     }
 
     @Override
-    public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Pair<LogicalVariable, LogicalVariable> pair)
-            throws AlgebricksException {
+    public Void visitNestedTupleSourceOperator(NestedTupleSourceOperator op,
+            Pair<LogicalVariable, LogicalVariable> pair) throws AlgebricksException {
         return null;
     }
 
@@ -218,8 +220,8 @@
     }
 
     @Override
-    public Void visitPartitioningSplitOperator(PartitioningSplitOperator op, Pair<LogicalVariable, LogicalVariable> pair)
-            throws AlgebricksException {
+    public Void visitPartitioningSplitOperator(PartitioningSplitOperator op,
+            Pair<LogicalVariable, LogicalVariable> pair) throws AlgebricksException {
         for (Mutable<ILogicalExpression> e : op.getExpressions()) {
             e.getValue().substituteVar(pair.first, pair.second);
         }
@@ -399,7 +401,7 @@
     }
 
     @Override
-    public Void visitInsertDeleteOperator(InsertDeleteOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+    public Void visitInsertOperator(InsertOperator op, Pair<LogicalVariable, LogicalVariable> pair)
             throws AlgebricksException {
         op.getPayloadExpression().getValue().substituteVar(pair.first, pair.second);
         for (Mutable<ILogicalExpression> e : op.getPrimaryKeyExpressions()) {
@@ -410,11 +412,21 @@
     }
 
     @Override
-    public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, Pair<LogicalVariable, LogicalVariable> pair)
+    public Void visitDeleteOperator(DeleteOperator op, Pair<LogicalVariable, LogicalVariable> pair)
             throws AlgebricksException {
         for (Mutable<ILogicalExpression> e : op.getPrimaryKeyExpressions()) {
             e.getValue().substituteVar(pair.first, pair.second);
         }
+        substVarTypes(op, pair);
+        return null;
+    }
+
+    @Override
+    public Void visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op,
+            Pair<LogicalVariable, LogicalVariable> pair) throws AlgebricksException {
+        for (Mutable<ILogicalExpression> e : op.getPrimaryKeyExpressions()) {
+            e.getValue().substituteVar(pair.first, pair.second);
+        }
         for (Mutable<ILogicalExpression> e : op.getSecondaryKeyExpressions()) {
             e.getValue().substituteVar(pair.first, pair.second);
         }
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index bfb9036..d3bd27a 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -22,7 +22,6 @@
 import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
@@ -34,6 +33,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
@@ -43,7 +43,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
@@ -74,7 +74,7 @@
 
 public class UsedVariableVisitor implements ILogicalOperatorVisitor<Void, Void> {
 
-    private Collection<LogicalVariable> usedVariables;
+    private final Collection<LogicalVariable> usedVariables;
 
     public UsedVariableVisitor(Collection<LogicalVariable> usedVariables) {
         this.usedVariables = usedVariables;
@@ -341,8 +341,21 @@
     }
 
     @Override
-    public Void visitInsertDeleteOperator(InsertDeleteOperator op, Void arg) {
+    public Void visitInsertOperator(InsertOperator op, Void arg) {
         op.getPayloadExpression().getValue().getUsedVariables(usedVariables);
+        for (Mutable<ILogicalExpression> e : op.getPrimaryKeyExpressions()) {
+            e.getValue().getUsedVariables(usedVariables);
+        }
+        if (op.getAdditionalFilteringExpressions() != null) {
+            for (Mutable<ILogicalExpression> e : op.getAdditionalFilteringExpressions()) {
+                e.getValue().getUsedVariables(usedVariables);
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Void visitDeleteOperator(DeleteOperator op, Void arg) {
         for (Mutable<ILogicalExpression> e : op.getPrimaryKeyExpressions()) {
             e.getValue().getUsedVariables(usedVariables);
         }
@@ -362,6 +375,9 @@
         for (Mutable<ILogicalExpression> e : op.getSecondaryKeyExpressions()) {
             e.getValue().getUsedVariables(usedVariables);
         }
+        if (op.getFilterExpression() != null) {
+            op.getFilterExpression().getValue().getUsedVariables(usedVariables);
+        }
         if (op.getAdditionalFilteringExpressions() != null) {
             for (Mutable<ILogicalExpression> e : op.getAdditionalFilteringExpressions()) {
                 e.getValue().getUsedVariables(usedVariables);
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
index 449c04f..1b8258b 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
@@ -35,8 +35,7 @@
 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.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.InsertOperator;
 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;
@@ -55,7 +54,7 @@
     private final IDataSource<?> dataSource;
 
     public BulkloadPOperator(LogicalVariable payload, List<LogicalVariable> keys,
-           List<LogicalVariable> additionalFilteringKeys, IDataSource<?> dataSource) {
+            List<LogicalVariable> additionalFilteringKeys, IDataSource<?> dataSource) {
         this.payload = payload;
         this.primaryKeys = keys;
         this.additionalFilteringKeys = additionalFilteringKeys;
@@ -73,8 +72,8 @@
         List<LogicalVariable> scanVariables = new ArrayList<>();
         scanVariables.addAll(primaryKeys);
         scanVariables.add(new LogicalVariable(-1));
-        IPhysicalPropertiesVector physicalProps = dataSource.getPropertiesProvider().computePropertiesVector(
-                scanVariables);
+        IPhysicalPropertiesVector physicalProps = dataSource.getPropertiesProvider()
+                .computePropertiesVector(scanVariables);
         StructuralPropertiesVector spv = new StructuralPropertiesVector(physicalProps.getPartitioningProperty(),
                 physicalProps.getLocalProperties());
         return new PhysicalRequirements(new IPhysicalPropertiesVector[] { spv },
@@ -91,9 +90,8 @@
     @Override
     public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
             IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
-            throws AlgebricksException {
-        InsertDeleteOperator insertDeleteOp = (InsertDeleteOperator) op;
-        assert insertDeleteOp.getOperation() == Kind.INSERT;
+                    throws AlgebricksException {
+        InsertOperator insertDeleteOp = (InsertOperator) op;
         assert insertDeleteOp.isBulkload();
 
         IMetadataProvider mp = context.getMetadataProvider();
@@ -101,9 +99,9 @@
         JobSpecification spec = builder.getJobSpec();
         RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(
                 context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0], context);
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = mp.getInsertRuntime(
-                dataSource, propagatedSchema, typeEnv, primaryKeys, payload, additionalFilteringKeys,
-                inputDesc, context, spec, true);
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = mp.getInsertRuntime(dataSource,
+                propagatedSchema, typeEnv, primaryKeys, payload, additionalFilteringKeys, inputDesc, context, spec,
+                true);
         builder.contributeHyracksOperator(insertDeleteOp, runtimeAndConstraints.first);
         builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
         ILogicalOperator src = insertDeleteOp.getInputs().get(0).getValue();
@@ -119,6 +117,5 @@
     public boolean expensiveThanMaterialization() {
         return false;
     }
-
 
 }
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/DeletePOperator.java
similarity index 79%
copy from algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
copy to algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DeletePOperator.java
index c774f44..9abbd47 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/DeletePOperator.java
@@ -33,9 +33,8 @@
 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.DeleteOperator;
 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.properties.IPartitioningRequirementsCoordinator;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
@@ -47,16 +46,14 @@
 import org.apache.hyracks.api.job.JobSpecification;
 
 @SuppressWarnings("rawtypes")
-public class InsertDeletePOperator extends AbstractPhysicalOperator {
+public class DeletePOperator extends AbstractPhysicalOperator {
 
-    private LogicalVariable payload;
     private List<LogicalVariable> keys;
     private IDataSource<?> dataSource;
     private final List<LogicalVariable> additionalFilteringKeys;
 
-    public InsertDeletePOperator(LogicalVariable payload, List<LogicalVariable> keys,
-            List<LogicalVariable> additionalFilteringKeys, IDataSource dataSource) {
-        this.payload = payload;
+    public DeletePOperator(List<LogicalVariable> keys, List<LogicalVariable> additionalFilteringKeys,
+            IDataSource dataSource) {
         this.keys = keys;
         this.dataSource = dataSource;
         this.additionalFilteringKeys = additionalFilteringKeys;
@@ -64,7 +61,7 @@
 
     @Override
     public PhysicalOperatorTag getOperatorTag() {
-        return PhysicalOperatorTag.INSERT_DELETE;
+        return PhysicalOperatorTag.DELETE;
     }
 
     @Override
@@ -90,27 +87,21 @@
     @Override
     public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
             IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
-            throws AlgebricksException {
-        InsertDeleteOperator insertDeleteOp = (InsertDeleteOperator) op;
+                    throws AlgebricksException {
+        DeleteOperator deleteOp = (DeleteOperator) op;
         IMetadataProvider mp = context.getMetadataProvider();
         IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
         JobSpecification spec = builder.getJobSpec();
         RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(
                 context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0], context);
 
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = null;
-        if (insertDeleteOp.getOperation() == Kind.INSERT) {
-            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);
-        }
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = mp.getDeleteRuntime(dataSource,
+                propagatedSchema, typeEnv, keys, additionalFilteringKeys, inputDesc, context, spec);
 
-        builder.contributeHyracksOperator(insertDeleteOp, runtimeAndConstraints.first);
+        builder.contributeHyracksOperator(deleteOp, runtimeAndConstraints.first);
         builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
-        ILogicalOperator src = insertDeleteOp.getInputs().get(0).getValue();
-        builder.contributeGraphEdge(src, 0, insertDeleteOp, 0);
+        ILogicalOperator src = deleteOp.getInputs().get(0).getValue();
+        builder.contributeGraphEdge(src, 0, deleteOp, 0);
     }
 
     @Override
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
index ff23703..e67c634 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexBulkloadPOperator.java
@@ -23,7 +23,6 @@
 import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -39,7 +38,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator.Kind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator.Kind;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
 import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
@@ -112,7 +111,7 @@
     @Override
     public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
             IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
-            throws AlgebricksException {
+                    throws AlgebricksException {
         IndexInsertDeleteOperator indexInsertDeleteOp = (IndexInsertDeleteOperator) op;
         assert indexInsertDeleteOp.getOperation() == Kind.INSERT;
         assert indexInsertDeleteOp.isBulkload();
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java
index 29673b4..bfbeba9 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/IndexInsertDeletePOperator.java
@@ -22,7 +22,6 @@
 import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -38,7 +37,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator.Kind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator.Kind;
 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;
@@ -100,7 +99,7 @@
     @Override
     public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
             IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
-            throws AlgebricksException {
+                    throws AlgebricksException {
         IndexInsertDeleteOperator insertDeleteOp = (IndexInsertDeleteOperator) op;
         IMetadataProvider mp = context.getMetadataProvider();
 
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/InsertPOperator.java
similarity index 85%
rename from algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertDeletePOperator.java
rename to algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertPOperator.java
index c774f44..4b865a7 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/InsertPOperator.java
@@ -34,8 +34,7 @@
 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.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.InsertOperator;
 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;
@@ -47,14 +46,14 @@
 import org.apache.hyracks.api.job.JobSpecification;
 
 @SuppressWarnings("rawtypes")
-public class InsertDeletePOperator extends AbstractPhysicalOperator {
+public class InsertPOperator extends AbstractPhysicalOperator {
 
     private LogicalVariable payload;
     private List<LogicalVariable> keys;
     private IDataSource<?> dataSource;
     private final List<LogicalVariable> additionalFilteringKeys;
 
-    public InsertDeletePOperator(LogicalVariable payload, List<LogicalVariable> keys,
+    public InsertPOperator(LogicalVariable payload, List<LogicalVariable> keys,
             List<LogicalVariable> additionalFilteringKeys, IDataSource dataSource) {
         this.payload = payload;
         this.keys = keys;
@@ -64,7 +63,7 @@
 
     @Override
     public PhysicalOperatorTag getOperatorTag() {
-        return PhysicalOperatorTag.INSERT_DELETE;
+        return PhysicalOperatorTag.INSERT;
     }
 
     @Override
@@ -90,22 +89,16 @@
     @Override
     public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
             IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
-            throws AlgebricksException {
-        InsertDeleteOperator insertDeleteOp = (InsertDeleteOperator) op;
+                    throws AlgebricksException {
+        InsertOperator insertDeleteOp = (InsertOperator) op;
         IMetadataProvider mp = context.getMetadataProvider();
         IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
         JobSpecification spec = builder.getJobSpec();
         RecordDescriptor inputDesc = JobGenHelper.mkRecordDescriptor(
                 context.getTypeEnvironment(op.getInputs().get(0).getValue()), inputSchemas[0], context);
 
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = null;
-        if (insertDeleteOp.getOperation() == Kind.INSERT) {
-            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);
-        }
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> runtimeAndConstraints = mp.getInsertRuntime(dataSource,
+                propagatedSchema, typeEnv, keys, payload, additionalFilteringKeys, inputDesc, context, spec, false);
 
         builder.contributeHyracksOperator(insertDeleteOp, runtimeAndConstraints.first);
         builder.contributeAlgebricksPartitionConstraint(runtimeAndConstraints.first, runtimeAndConstraints.second);
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/TokenizePOperator.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/TokenizePOperator.java
index e1983d2..c7de0a5 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/TokenizePOperator.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/TokenizePOperator.java
@@ -19,7 +19,6 @@
 
 package org.apache.hyracks.algebricks.core.algebra.operators.physical;
 
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -35,7 +34,7 @@
 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.IOperatorSchema;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator.Kind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator.Kind;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOperator;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
@@ -80,7 +79,7 @@
     @Override
     public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
             IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
-            throws AlgebricksException {
+                    throws AlgebricksException {
         TokenizeOperator tokenizeOp = (TokenizeOperator) op;
         if (tokenizeOp.getOperation() != Kind.INSERT || !tokenizeOp.isBulkload()) {
             throw new AlgebricksException("Tokenize Operator only works when bulk-loading data.");
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
index 1404a47..7335ef3 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
@@ -21,7 +21,6 @@
 import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
@@ -32,6 +31,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
@@ -40,9 +40,9 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExternalDataLookupOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator.Kind;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
-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.InsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
@@ -86,7 +86,8 @@
     }
 
     @Override
-    public String visitRunningAggregateOperator(RunningAggregateOperator op, Integer indent) throws AlgebricksException {
+    public String visitRunningAggregateOperator(RunningAggregateOperator op, Integer indent)
+            throws AlgebricksException {
         StringBuilder buffer = new StringBuilder();
         addIndent(buffer, indent).append("running-aggregate ").append(op.getVariables()).append(" <- ");
         pprintExprList(op.getExpressions(), buffer, indent);
@@ -185,7 +186,8 @@
     }
 
     @Override
-    public String visitDistributeResultOperator(DistributeResultOperator op, Integer indent) throws AlgebricksException {
+    public String visitDistributeResultOperator(DistributeResultOperator op, Integer indent)
+            throws AlgebricksException {
         StringBuilder buffer = new StringBuilder();
         addIndent(buffer, indent).append("distribute result ");
         pprintExprList(op.getExpressions(), buffer, indent);
@@ -258,9 +260,8 @@
     @Override
     public String visitUnnestMapOperator(UnnestMapOperator op, Integer indent) throws AlgebricksException {
         StringBuilder buffer = new StringBuilder();
-        addIndent(buffer, indent).append(
-                "unnest-map " + op.getVariables() + " <- "
-                        + op.getExpressionRef().getValue().accept(exprVisitor, indent));
+        addIndent(buffer, indent).append("unnest-map " + op.getVariables() + " <- "
+                + op.getExpressionRef().getValue().accept(exprVisitor, indent));
         return buffer.toString();
     }
 
@@ -293,8 +294,8 @@
     @Override
     public String visitScriptOperator(ScriptOperator op, Integer indent) {
         StringBuilder buffer = new StringBuilder();
-        addIndent(buffer, indent).append(
-                "script (in: " + op.getInputVariables() + ") (out: " + op.getOutputVariables() + ")");
+        addIndent(buffer, indent)
+                .append("script (in: " + op.getInputVariables() + ") (out: " + op.getOutputVariables() + ")");
         return buffer.toString();
     }
 
@@ -313,14 +314,30 @@
     }
 
     @Override
-    public String visitInsertDeleteOperator(InsertDeleteOperator op, Integer indent) throws AlgebricksException {
+    public String visitInsertOperator(InsertOperator op, Integer indent) throws AlgebricksException {
         StringBuilder buffer = new StringBuilder();
-        String header = op.getOperation() == Kind.INSERT ? "insert into " : "delete from ";
-        addIndent(buffer, indent).append(header).append(op.getDataSource()).append(" from ")
-                .append(op.getPayloadExpression().getValue().accept(exprVisitor, indent)).append(" partitioned by ");
+        addIndent(buffer, indent).append("insert ")
+                .append(op.getPayloadExpression().getValue().accept(exprVisitor, indent)).append(" into ")
+                .append(op.getDataSource()).append(" partitioned by ");
         pprintExprList(op.getPrimaryKeyExpressions(), buffer, indent);
+        if (op.getAdditionalFilteringExpressions() != null) {
+            buffer.append(" filtered by ");
+            pprintExprList(op.getAdditionalFilteringExpressions(), buffer, indent);
+        }
         if (op.isBulkload()) {
             buffer.append(" [bulkload]");
+        }
+        return buffer.toString();
+    }
+
+    @Override
+    public String visitDeleteOperator(DeleteOperator op, Integer indent) throws AlgebricksException {
+        StringBuilder buffer = new StringBuilder();
+        addIndent(buffer, indent).append("delete from ").append(op.getDataSource()).append(" partitioned by ");
+        pprintExprList(op.getPrimaryKeyExpressions(), buffer, indent);
+        if (op.getAdditionalFilteringExpressions() != null) {
+            buffer.append(" filtered by ");
+            pprintExprList(op.getAdditionalFilteringExpressions(), buffer, indent);
         }
         return buffer.toString();
     }
@@ -329,10 +346,11 @@
     public String visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, Integer indent)
             throws AlgebricksException {
         StringBuilder buffer = new StringBuilder();
-        String header = op.getOperation() == Kind.INSERT ? "insert into " : "delete from ";
-        addIndent(buffer, indent).append(header).append(op.getIndexName()).append(" on ")
-                .append(op.getDataSourceIndex().getDataSource()).append(" from ");
+        addIndent(buffer, indent).append(op.getOperation() == Kind.INSERT ? "insert " : "delete ");
         pprintExprList(op.getSecondaryKeyExpressions(), buffer, indent);
+        buffer.append(op.getOperation() == Kind.INSERT ? " into " : " from ").append(op.getIndexName()).append(" on ")
+                .append(op.getDataSourceIndex().getDataSource()).append(" partitioned by ");
+        pprintExprList(op.getPrimaryKeyExpressions(), buffer, indent);
         if (op.isBulkload()) {
             buffer.append(" [bulkload]");
         }
@@ -427,8 +445,8 @@
     public String visitExternalDataLookupOperator(ExternalDataLookupOperator op, Integer indent)
             throws AlgebricksException {
         StringBuilder buffer = new StringBuilder();
-        addIndent(buffer, indent).append(
-                "external-instant-lookup " + op.getVariables() + " <- " + op.getExpressionRef().getValue());
+        addIndent(buffer, indent)
+                .append("external-instant-lookup " + op.getVariables() + " <- " + op.getExpressionRef().getValue());
         return buffer.toString();
     }
 
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/PlanPrettyPrinter.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/PlanPrettyPrinter.java
index 50e0a00..60c37a0 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/PlanPrettyPrinter.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/PlanPrettyPrinter.java
@@ -16,79 +16,89 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.hyracks.algebricks.core.algebra.prettyprint;
-
-import org.apache.commons.lang3.mutable.Mutable;
-
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
-import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
-
-public class PlanPrettyPrinter {
-    public static void printPlan(ILogicalPlan plan, StringBuilder out, LogicalOperatorPrettyPrintVisitor pvisitor,
-            int indent) throws AlgebricksException {
-        for (Mutable<ILogicalOperator> root : plan.getRoots()) {
-            printOperator((AbstractLogicalOperator) root.getValue(), out, pvisitor, indent);
-        }
-    }
-
-    public static void printPhysicalOps(ILogicalPlan plan, StringBuilder out, int indent) {
-        for (Mutable<ILogicalOperator> root : plan.getRoots()) {
-            printPhysicalOperator((AbstractLogicalOperator) root.getValue(), indent, out);
-        }
-    }
-
-    public static void printOperator(AbstractLogicalOperator op, StringBuilder out,
-            LogicalOperatorPrettyPrintVisitor pvisitor, int indent) throws AlgebricksException {
-        out.append(op.accept(pvisitor, indent));
-        IPhysicalOperator pOp = op.getPhysicalOperator();
-
-        if (pOp != null) {
-            out.append("\n");
-            pad(out, indent);
-            appendln(out, "-- " + pOp.toString() + "  |" + op.getExecutionMode() + "|");
-        } else {
-            appendln(out, " -- |" + op.getExecutionMode() + "|");
-        }
-
-        for (Mutable<ILogicalOperator> i : op.getInputs()) {
-            printOperator((AbstractLogicalOperator) i.getValue(), out, pvisitor, indent + 2);
-        }
-
-    }
-
-    public static void printPhysicalOperator(AbstractLogicalOperator op, int indent, StringBuilder out) {
-        IPhysicalOperator pOp = op.getPhysicalOperator();
-        pad(out, indent);
-        appendln(out, "-- " + pOp.toString() + "  |" + op.getExecutionMode() + "|");
-        if (op.hasNestedPlans()) {
-            AbstractOperatorWithNestedPlans opNest = (AbstractOperatorWithNestedPlans) op;
-            for (ILogicalPlan p : opNest.getNestedPlans()) {
-                pad(out, indent + 8);
-                appendln(out, "{");
-                printPhysicalOps(p, out, indent + 10);
-                pad(out, indent + 8);
-                appendln(out, "}");
-            }
-        }
-
-        for (Mutable<ILogicalOperator> i : op.getInputs()) {
-            printPhysicalOperator((AbstractLogicalOperator) i.getValue(), indent + 2, out);
-        }
-
-    }
-
-    private static void appendln(StringBuilder buf, String s) {
-        buf.append(s);
-        buf.append("\n");
-    }
-
-    private static void pad(StringBuilder buf, int indent) {
-        for (int i = 0; i < indent; ++i) {
-            buf.append(' ');
-        }
-    }
-}
+package org.apache.hyracks.algebricks.core.algebra.prettyprint;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+
+public class PlanPrettyPrinter {
+    public static void printPlan(ILogicalPlan plan, StringBuilder out, LogicalOperatorPrettyPrintVisitor pvisitor,
+            int indent) throws AlgebricksException {
+        Map<AbstractLogicalOperator, Boolean> printedOpMap = new HashMap<>();
+        for (Mutable<ILogicalOperator> root : plan.getRoots()) {
+            printOperator((AbstractLogicalOperator) root.getValue(), out, pvisitor, indent, printedOpMap);
+        }
+    }
+
+    public static void printPhysicalOps(ILogicalPlan plan, StringBuilder out, int indent) {
+        Map<AbstractLogicalOperator, Boolean> printedOpMap = new HashMap<>();
+        for (Mutable<ILogicalOperator> root : plan.getRoots()) {
+            printPhysicalOperator((AbstractLogicalOperator) root.getValue(), indent, out, printedOpMap);
+        }
+    }
+
+    public static void printOperator(AbstractLogicalOperator op, StringBuilder out,
+            LogicalOperatorPrettyPrintVisitor pvisitor, int indent, Map<AbstractLogicalOperator, Boolean> printedOpMap)
+                    throws AlgebricksException {
+        IPhysicalOperator pOp = op.getPhysicalOperator();
+        out.append(op.accept(pvisitor, indent));
+        if (pOp != null) {
+            out.append("\n");
+            pad(out, indent);
+            appendln(out, "-- " + pOp.toString() + "  |" + op.getExecutionMode() + "|");
+        } else {
+            appendln(out, " -- |" + op.getExecutionMode() + "|");
+        }
+        Boolean opAlreadyPrinted = printedOpMap.put(op, true);
+        if (opAlreadyPrinted == null || !opAlreadyPrinted) {
+            for (Mutable<ILogicalOperator> i : op.getInputs()) {
+                printOperator((AbstractLogicalOperator) i.getValue(), out, pvisitor, indent + 2, printedOpMap);
+            }
+        }
+
+    }
+
+    public static void printPhysicalOperator(AbstractLogicalOperator op, int indent, StringBuilder out,
+            Map<AbstractLogicalOperator, Boolean> printedOpMap) {
+        IPhysicalOperator pOp = op.getPhysicalOperator();
+        pad(out, indent);
+        appendln(out, "-- " + pOp.toString() + "  |" + op.getExecutionMode() + "|");
+        if (op.hasNestedPlans()) {
+            AbstractOperatorWithNestedPlans opNest = (AbstractOperatorWithNestedPlans) op;
+            for (ILogicalPlan p : opNest.getNestedPlans()) {
+                pad(out, indent + 8);
+                appendln(out, "{");
+                printPhysicalOps(p, out, indent + 10);
+                pad(out, indent + 8);
+                appendln(out, "}");
+            }
+        }
+
+        Boolean opAlreadyPrinted = printedOpMap.put(op, true);
+        if (opAlreadyPrinted == null || !opAlreadyPrinted) {
+            for (Mutable<ILogicalOperator> i : op.getInputs()) {
+                printPhysicalOperator((AbstractLogicalOperator) i.getValue(), indent + 2, out, printedOpMap);
+            }
+        }
+
+    }
+
+    private static void appendln(StringBuilder buf, String s) {
+        buf.append(s);
+        buf.append("\n");
+    }
+
+    private static void pad(StringBuilder buf, int indent) {
+        for (int i = 0; i < indent; ++i) {
+            buf.append(' ');
+        }
+    }
+}
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
index 423a0a4..89df3ef 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
@@ -22,6 +22,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
@@ -31,7 +32,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
@@ -110,7 +111,9 @@
 
     public R visitWriteResultOperator(WriteResultOperator op, T arg) throws AlgebricksException;
 
-    public R visitInsertDeleteOperator(InsertDeleteOperator op, T tag) throws AlgebricksException;
+    public R visitInsertOperator(InsertOperator op, T tag) throws AlgebricksException;
+
+    public R visitDeleteOperator(DeleteOperator op, T tag) throws AlgebricksException;
 
     public R visitIndexInsertDeleteOperator(IndexInsertDeleteOperator op, T tag) throws AlgebricksException;
 
diff --git a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java
index e3695fd..fd836cf 100644
--- a/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java
+++ b/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java
@@ -19,10 +19,10 @@
 package org.apache.hyracks.algebricks.core.rewriter.base;
 
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.logging.Level;
 
 import org.apache.commons.lang3.mutable.Mutable;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
@@ -69,7 +69,8 @@
         if (AlgebricksConfig.ALGEBRICKS_LOGGER.isLoggable(Level.FINE)) {
             StringBuilder sb = new StringBuilder();
             LogicalOperatorPrettyPrintVisitor pvisitor = context.getPrettyPrintVisitor();
-            PlanPrettyPrinter.printOperator((AbstractLogicalOperator) opRef.getValue(), sb, pvisitor, 0);
+            PlanPrettyPrinter.printOperator((AbstractLogicalOperator) opRef.getValue(), sb, pvisitor, 0,
+                    new HashMap<AbstractLogicalOperator, Boolean>());
             return sb.toString();
         }
         return null;
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..0fdf505 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
@@ -84,7 +84,7 @@
             List<LogicalVariable> scanVariables, List<LogicalVariable> projectVariables, boolean projectPushed,
             List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
-            throws AlgebricksException {
+                    throws AlgebricksException {
         PigletFileDataSource ds = (PigletFileDataSource) dataSource;
 
         FileSplit[] fileSplits = ds.getFileSplits();
@@ -141,15 +141,15 @@
     @Override
     public Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint> getWriteFileRuntime(IDataSink sink,
             int[] printColumns, IPrinterFactory[] printerFactories, RecordDescriptor inputDesc)
-            throws AlgebricksException {
+                    throws AlgebricksException {
         PigletFileDataSink ds = (PigletFileDataSink) sink;
         FileSplit[] fileSplits = ds.getFileSplits();
         String[] locations = new String[fileSplits.length];
         for (int i = 0; i < fileSplits.length; ++i) {
             locations[i] = fileSplits[i].getNodeName();
         }
-        IPushRuntimeFactory prf = new SinkWriterRuntimeFactory(printColumns, printerFactories, fileSplits[0]
-                .getLocalFile().getFile(), PrinterBasedWriterFactory.INSTANCE, inputDesc);
+        IPushRuntimeFactory prf = new SinkWriterRuntimeFactory(printColumns, printerFactories,
+                fileSplits[0].getLocalFile().getFile(), PrinterBasedWriterFactory.INSTANCE, inputDesc);
         AlgebricksAbsolutePartitionConstraint constraint = new AlgebricksAbsolutePartitionConstraint(locations);
         return new Pair<IPushRuntimeFactory, AlgebricksPartitionConstraint>(prf, constraint);
     }
@@ -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;
     }
@@ -197,16 +193,16 @@
             IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, List<LogicalVariable> additionalNonKeyFields,
             ILogicalExpression filterExpr, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
-            throws AlgebricksException {
+                    throws AlgebricksException {
         // TODO Auto-generated method stub
         return null;
     }
 
     @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;
@@ -229,8 +225,15 @@
     @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) throws AlgebricksException {
+            List<LogicalVariable> additionalNonKeyFields, RecordDescriptor recordDesc, JobGenContext context,
+            JobSpecification jobSpec) throws AlgebricksException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
+    public IDataSourceIndex<String, String> findIndex(String indexId, String dataverseName, String datasetName)
+            throws AlgebricksException {
         // TODO Auto-generated method stub
         return null;
     }
diff --git a/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java b/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
index 4df9db7..08754c9 100644
--- a/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
@@ -21,13 +21,13 @@
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -105,7 +105,8 @@
     }
 
     @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
         // wait for the physical operators to be set first
         if (op.getPhysicalOperator() == null) {
@@ -207,8 +208,8 @@
             AbstractLogicalOperator child = (AbstractLogicalOperator) childRef.getValue();
             IPhysicalPropertiesVector delivered = child.getDeliveredPhysicalProperties();
 
-            AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Properties delivered by " + child.getPhysicalOperator()
-                    + ": " + delivered + "\n");
+            AlgebricksConfig.ALGEBRICKS_LOGGER
+                    .finest(">>>> Properties delivered by " + child.getPhysicalOperator() + ": " + delivered + "\n");
             IPartitioningRequirementsCoordinator prc = pr.getPartitioningCoordinator();
             Pair<Boolean, IPartitioningProperty> pbpp = prc.coordinateRequirements(
                     reqdProperties[i].getPartitioningProperty(), firstDeliveredPartitioning, op, context);
@@ -216,8 +217,8 @@
             IPhysicalPropertiesVector rqd = new StructuralPropertiesVector(pbpp.second,
                     reqdProperties[i].getLocalProperties());
 
-            AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Required properties for " + child.getPhysicalOperator()
-                    + ": " + rqd + "\n");
+            AlgebricksConfig.ALGEBRICKS_LOGGER
+                    .finest(">>>> Required properties for " + child.getPhysicalOperator() + ": " + rqd + "\n");
             IPhysicalPropertiesVector diff = delivered.getUnsatisfiedPropertiesFrom(rqd,
                     mayExpandPartitioningProperties, context.getEquivalenceClassMap(child), context.getFDList(child));
 
@@ -266,8 +267,8 @@
 
         if (opIsRedundantSort) {
             if (AlgebricksConfig.DEBUG) {
-                AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Removing redundant SORT operator "
-                        + op.getPhysicalOperator() + "\n");
+                AlgebricksConfig.ALGEBRICKS_LOGGER
+                        .fine(">>>> Removing redundant SORT operator " + op.getPhysicalOperator() + "\n");
                 printOp(op);
             }
             changed = true;
@@ -291,7 +292,7 @@
 
     private IPhysicalPropertiesVector newPropertiesDiff(AbstractLogicalOperator newChild,
             IPhysicalPropertiesVector required, boolean mayExpandPartitioningProperties, IOptimizationContext context)
-            throws AlgebricksException {
+                    throws AlgebricksException {
         IPhysicalPropertiesVector newDelivered = newChild.getDeliveredPhysicalProperties();
 
         Map<LogicalVariable, EquivalenceClass> newChildEqClasses = context.getEquivalenceClassMap(newChild);
@@ -302,8 +303,8 @@
             newChildEqClasses = context.getEquivalenceClassMap(newChild);
             newChildFDs = context.getFDList(newChild);
         }
-        AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Required properties for new op. "
-                + newChild.getPhysicalOperator() + ": " + required + "\n");
+        AlgebricksConfig.ALGEBRICKS_LOGGER.finest(
+                ">>>> Required properties for new op. " + newChild.getPhysicalOperator() + ": " + required + "\n");
 
         return newDelivered.getUnsatisfiedPropertiesFrom(required, mayExpandPartitioningProperties, newChildEqClasses,
                 newChildFDs);
@@ -377,16 +378,16 @@
             IPhysicalPropertiesVector diffOfProperties, IOptimizationContext context) {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
         if (op.getOperatorTag() != LogicalOperatorTag.ORDER
-                || (op.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.STABLE_SORT && op
-                        .getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.IN_MEMORY_STABLE_SORT)
+                || (op.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.STABLE_SORT
+                        && op.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.IN_MEMORY_STABLE_SORT)
                 || delivered.getLocalProperties() == null) {
             return false;
         }
         AbstractStableSortPOperator sortOp = (AbstractStableSortPOperator) op.getPhysicalOperator();
         sortOp.computeLocalProperties(op);
         ILocalStructuralProperty orderProp = sortOp.getOrderProperty();
-        return PropertiesUtil.matchLocalProperties(Collections.singletonList(orderProp),
-                delivered.getLocalProperties(), context.getEquivalenceClassMap(op), context.getFDList(op));
+        return PropertiesUtil.matchLocalProperties(Collections.singletonList(orderProp), delivered.getLocalProperties(),
+                context.getEquivalenceClassMap(op), context.getFDList(op));
     }
 
     private void addEnforcers(AbstractLogicalOperator op, int childIndex,
@@ -414,8 +415,8 @@
     private void addLocalEnforcers(AbstractLogicalOperator op, int i, List<ILocalStructuralProperty> localProperties,
             boolean nestedPlan, IOptimizationContext context) throws AlgebricksException {
         if (AlgebricksConfig.DEBUG) {
-            AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Adding local enforcers for local props = " + localProperties
-                    + "\n");
+            AlgebricksConfig.ALGEBRICKS_LOGGER
+                    .fine(">>>> Adding local enforcers for local props = " + localProperties + "\n");
         }
 
         if (localProperties == null || localProperties.isEmpty()) {
@@ -434,8 +435,8 @@
                 }
                 case LOCAL_GROUPING_PROPERTY: {
                     LocalGroupingProperty g = (LocalGroupingProperty) prop;
-                    Collection<LogicalVariable> vars = (g.getPreferredOrderEnforcer() != null) ? g
-                            .getPreferredOrderEnforcer() : g.getColumnSet();
+                    Collection<LogicalVariable> vars = (g.getPreferredOrderEnforcer() != null)
+                            ? g.getPreferredOrderEnforcer() : g.getColumnSet();
                     List<OrderColumn> orderColumns = new ArrayList<OrderColumn>();
                     for (LogicalVariable v : vars) {
                         OrderColumn oc = new OrderColumn(v, OrderKind.ASC);
@@ -461,7 +462,7 @@
 
     private Mutable<ILogicalOperator> enforceOrderProperties(List<LocalOrderProperty> oList,
             Mutable<ILogicalOperator> topOp, boolean isMicroOp, IOptimizationContext context)
-            throws AlgebricksException {
+                    throws AlgebricksException {
         List<Pair<IOrder, Mutable<ILogicalExpression>>> oe = new LinkedList<Pair<IOrder, Mutable<ILogicalExpression>>>();
         for (LocalOrderProperty orderProperty : oList) {
             for (OrderColumn oc : orderProperty.getOrderColumns()) {
@@ -498,8 +499,8 @@
                         pop = new RandomMergeExchangePOperator();
                     } else {
                         if (op.getAnnotations().containsKey(OperatorAnnotations.USE_RANGE_CONNECTOR)) {
-                            IRangeMap rangeMap = (IRangeMap) op.getAnnotations().get(
-                                    OperatorAnnotations.USE_RANGE_CONNECTOR);
+                            IRangeMap rangeMap = (IRangeMap) op.getAnnotations()
+                                    .get(OperatorAnnotations.USE_RANGE_CONNECTOR);
                             pop = new RangePartitionMergePOperator(ordCols, domain, rangeMap);
                         } else {
                             OrderColumn[] sortColumns = new OrderColumn[ordCols.size()];
@@ -533,7 +534,8 @@
                     break;
                 }
                 case ORDERED_PARTITIONED: {
-                    pop = new RangePartitionPOperator(((OrderedPartitionedProperty) pp).getOrderColumns(), domain, null);
+                    pop = new RangePartitionPOperator(((OrderedPartitionedProperty) pp).getOrderColumns(), domain,
+                            null);
                     break;
                 }
                 case BROADCAST: {
@@ -559,8 +561,8 @@
             OperatorPropertiesUtil.computeSchemaAndPropertiesRecIfNull(exchg, context);
             context.computeAndSetTypeEnvironmentForOperator(exchg);
             if (AlgebricksConfig.DEBUG) {
-                AlgebricksConfig.ALGEBRICKS_LOGGER.fine(">>>> Added partitioning enforcer "
-                        + exchg.getPhysicalOperator() + ".\n");
+                AlgebricksConfig.ALGEBRICKS_LOGGER
+                        .fine(">>>> Added partitioning enforcer " + exchg.getPhysicalOperator() + ".\n");
                 printOp((AbstractLogicalOperator) op);
             }
         }
@@ -578,7 +580,7 @@
     private void printOp(AbstractLogicalOperator op) throws AlgebricksException {
         StringBuilder sb = new StringBuilder();
         LogicalOperatorPrettyPrintVisitor pvisitor = new LogicalOperatorPrettyPrintVisitor();
-        PlanPrettyPrinter.printOperator(op, sb, pvisitor, 0);
+        PlanPrettyPrinter.printOperator(op, sb, pvisitor, 0, new HashMap<AbstractLogicalOperator, Boolean>());
         AlgebricksConfig.ALGEBRICKS_LOGGER.fine(sb.toString());
     }
 
@@ -609,8 +611,8 @@
         newOp.recomputeSchema();
         newOp.computeDeliveredPhysicalProperties(context);
         context.computeAndSetTypeEnvironmentForOperator(newOp);
-        AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Structural properties for " + newOp.getPhysicalOperator()
-                + ": " + newOp.getDeliveredPhysicalProperties() + "\n");
+        AlgebricksConfig.ALGEBRICKS_LOGGER.finest(">>>> Structural properties for " + newOp.getPhysicalOperator() + ": "
+                + newOp.getDeliveredPhysicalProperties() + "\n");
 
         PhysicalOptimizationsUtil.computeFDsAndEquivalenceClasses(newOp, context);
     }
diff --git a/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java b/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
index 394524b..d3db98f 100644
--- a/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
+++ b/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
@@ -23,7 +23,6 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -43,11 +42,12 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.DeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.InsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
@@ -57,13 +57,14 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.AssignPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.BulkloadPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.DataSourceScanPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.DeletePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.DistributeResultPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.EmptyTupleSourcePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.ExternalGroupByPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.InMemoryStableSortPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.IndexBulkloadPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.IndexInsertDeletePOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.physical.InsertDeletePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.InsertPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.MicroPreclusteredGroupByPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.NestedTupleSourcePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.PreSortedDistinctByPOperator;
@@ -95,7 +96,8 @@
     }
 
     @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
         // if (context.checkIfInDontApplySet(this, op)) {
         // return false;
@@ -152,8 +154,8 @@
                     if (gby.getNestedPlans().size() == 1) {
                         ILogicalPlan p0 = gby.getNestedPlans().get(0);
                         if (p0.getRoots().size() == 1) {
-                            if (gby.getAnnotations().get(OperatorAnnotations.USE_HASH_GROUP_BY) == Boolean.TRUE
-                                    || gby.getAnnotations().get(OperatorAnnotations.USE_EXTERNAL_GROUP_BY) == Boolean.TRUE) {
+                            if (gby.getAnnotations().get(OperatorAnnotations.USE_HASH_GROUP_BY) == Boolean.TRUE || gby
+                                    .getAnnotations().get(OperatorAnnotations.USE_EXTERNAL_GROUP_BY) == Boolean.TRUE) {
                                 if (!topLevelOp) {
                                     throw new NotImplementedException(
                                             "External hash group-by for nested grouping is not implemented.");
@@ -213,8 +215,8 @@
                         }
                     }
                     if (topLevelOp) {
-                        op.setPhysicalOperator(new StableSortPOperator(physicalOptimizationConfig
-                                .getMaxFramesExternalSort()));
+                        op.setPhysicalOperator(
+                                new StableSortPOperator(physicalOptimizationConfig.getMaxFramesExternalSort()));
                     } else {
                         op.setPhysicalOperator(new InMemoryStableSortPOperator());
                     }
@@ -282,12 +284,12 @@
                         additionalFilteringKeys = new ArrayList<LogicalVariable>();
                         getKeys(opLoad.getAdditionalFilteringExpressions(), additionalFilteringKeys);
                     }
-                    op.setPhysicalOperator(new WriteResultPOperator(opLoad.getDataSource(), payload, keys,
-                            additionalFilteringKeys));
+                    op.setPhysicalOperator(
+                            new WriteResultPOperator(opLoad.getDataSource(), payload, keys, additionalFilteringKeys));
                     break;
                 }
-                case INSERT_DELETE: {
-                    InsertDeleteOperator opLoad = (InsertDeleteOperator) op;
+                case INSERT: {
+                    InsertOperator opLoad = (InsertOperator) op;
                     LogicalVariable payload;
                     List<LogicalVariable> keys = new ArrayList<LogicalVariable>();
                     List<LogicalVariable> additionalFilteringKeys = null;
@@ -297,12 +299,24 @@
                         getKeys(opLoad.getAdditionalFilteringExpressions(), additionalFilteringKeys);
                     }
                     if (opLoad.isBulkload()) {
-                        op.setPhysicalOperator(new BulkloadPOperator(payload, keys, additionalFilteringKeys, opLoad
-                                .getDataSource()));
+                        op.setPhysicalOperator(
+                                new BulkloadPOperator(payload, keys, additionalFilteringKeys, opLoad.getDataSource()));
                     } else {
-                        op.setPhysicalOperator(new InsertDeletePOperator(payload, keys, additionalFilteringKeys, opLoad
-                                .getDataSource()));
+                        op.setPhysicalOperator(
+                                new InsertPOperator(payload, keys, additionalFilteringKeys, opLoad.getDataSource()));
                     }
+                    break;
+                }
+                case DELETE: {
+                    DeleteOperator opLoad = (DeleteOperator) op;
+                    List<LogicalVariable> keys = new ArrayList<LogicalVariable>();
+                    List<LogicalVariable> additionalFilteringKeys = null;
+                    getKeys(opLoad.getPrimaryKeyExpressions(), keys);
+                    if (opLoad.getAdditionalFilteringExpressions() != null) {
+                        additionalFilteringKeys = new ArrayList<LogicalVariable>();
+                        getKeys(opLoad.getAdditionalFilteringExpressions(), additionalFilteringKeys);
+                    }
+                    op.setPhysicalOperator(new DeletePOperator(keys, additionalFilteringKeys, opLoad.getDataSource()));
                     break;
                 }
                 case INDEX_INSERT_DELETE: {
@@ -317,11 +331,13 @@
                         getKeys(opInsDel.getAdditionalFilteringExpressions(), additionalFilteringKeys);
                     }
                     if (opInsDel.isBulkload()) {
-                        op.setPhysicalOperator(new IndexBulkloadPOperator(primaryKeys, secondaryKeys,
-                                additionalFilteringKeys, opInsDel.getFilterExpression(), opInsDel.getDataSourceIndex()));
+                        op.setPhysicalOperator(
+                                new IndexBulkloadPOperator(primaryKeys, secondaryKeys, additionalFilteringKeys,
+                                        opInsDel.getFilterExpression(), opInsDel.getDataSourceIndex()));
                     } else {
-                        op.setPhysicalOperator(new IndexInsertDeletePOperator(primaryKeys, secondaryKeys,
-                                additionalFilteringKeys, opInsDel.getFilterExpression(), opInsDel.getDataSourceIndex()));
+                        op.setPhysicalOperator(
+                                new IndexInsertDeletePOperator(primaryKeys, secondaryKeys, additionalFilteringKeys,
+                                        opInsDel.getFilterExpression(), opInsDel.getDataSourceIndex()));
                     }
 
                     break;
@@ -335,8 +351,8 @@
                     getKeys(opTokenize.getSecondaryKeyExpressions(), secondaryKeys);
                     // Tokenize Operator only operates with a bulk load on a data set with an index
                     if (opTokenize.isBulkload()) {
-                        op.setPhysicalOperator(new TokenizePOperator(primaryKeys, secondaryKeys, opTokenize
-                                .getDataSourceIndex()));
+                        op.setPhysicalOperator(
+                                new TokenizePOperator(primaryKeys, secondaryKeys, opTokenize.getDataSourceIndex()));
                     }
                     break;
                 }
@@ -415,8 +431,8 @@
         int n = aggOp.getExpressions().size();
         List<Mutable<ILogicalExpression>> mergeExpressionRefs = new ArrayList<Mutable<ILogicalExpression>>();
         for (int i = 0; i < n; i++) {
-            ILogicalExpression mergeExpr = mergeAggregationExpressionFactory.createMergeAggregation(
-                    originalAggVars.get(i), aggFuncRefs.get(i).getValue(), context);
+            ILogicalExpression mergeExpr = mergeAggregationExpressionFactory
+                    .createMergeAggregation(originalAggVars.get(i), aggFuncRefs.get(i).getValue(), context);
             if (mergeExpr == null) {
                 return false;
             }
diff --git a/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java b/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
index 9c8a35c..8108d6f 100644
--- a/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
+++ b/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
@@ -38,8 +38,8 @@
 
     private static final long serialVersionUID = 1L;
 
-    private int[] outColumns;
-    private IScalarEvaluatorFactory[] evalFactories;
+    private final int[] outColumns;
+    private final IScalarEvaluatorFactory[] evalFactories;
     private final boolean flushFramesRapidly;
 
     /**
@@ -92,9 +92,9 @@
         }
 
         return new AbstractOneInputOneOutputOneFramePushRuntime() {
-            private IPointable result = VoidPointable.FACTORY.createPointable();
-            private IScalarEvaluator[] eval = new IScalarEvaluator[evalFactories.length];
-            private ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(projectionList.length);
+            private final IPointable result = VoidPointable.FACTORY.createPointable();
+            private final IScalarEvaluator[] eval = new IScalarEvaluator[evalFactories.length];
+            private final ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(projectionList.length);
             private boolean first = true;
 
             @Override

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

Gerrit-MessageType: newchange
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 1
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 2: Verified+1

Build Successful 

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 2
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 3: Verified+1

Build Successful 

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 3
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 4:

Yingyi encountered the same issue as well and reported it on ASTERIXDB-1137

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 2
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 4: Verified-1

Build Failed 

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 4: Verified-1

Build Failed 

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 4:

It's either not deterministic or it's something specific to this change. Preston's change is based on the same parent and built fine.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

Posted by "Ildar Absalyamov (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/438

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................

ASTERIXDB-1109: Fixed deletion of records from open secondary index

- Separated InsertDeleteOperator into two distinct operators
- Added a rule, which reuses PK vars, retrieved from datascan\idxLookup
- Changed plan printer for insert\delete\idxInsert\idxDelete operators
- Modified plan printer to skip repeated parts of the plan under replicate operator

Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractModificationOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DeleteOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/TokenizeOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
C algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DeletePOperator.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/IndexInsertDeletePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/TokenizePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/PlanPrettyPrinter.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.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
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
M algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
32 files changed, 578 insertions(+), 377 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 2
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

Posted by "Ildar Absalyamov (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/438

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................

ASTERIXDB-1109: Fixed deletion of records from open secondary index

- Separated InsertDeleteOperator into two distinct operators
- Added a rule, which reuses PK vars, retrieved from datascan\idxLookup
- Changed plan printer for insert\delete\idxInsert\idxDelete operators
- Modified plan printer to skip repeated parts of the plan under replicate operator

Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractModificationOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DeleteOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/TokenizeOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
C algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DeletePOperator.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/IndexInsertDeletePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/TokenizePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/PlanPrettyPrinter.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.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
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
M algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
32 files changed, 587 insertions(+), 385 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 1: Verified+1

Build Successful 

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 1
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 4: Code-Review-2

Similarly, this needs to go on release-0.2.18

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 3:

(6 comments)

https://asterix-gerrit.ics.uci.edu/#/c/438/3/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java:

Line 32:     INDEX_INSERT_DELETE,
> What's the main difference between index insert/delete and insert/delete? I
The main reasoning was that dataset insert & dataset delete operations are semantically different: insert requires having a PK and a record, whereas delete work while having only PK. This logic does not apply to index insert/delete, since there we need SK&PK in both cases, hence I kept those operations together


https://asterix-gerrit.ics.uci.edu/#/c/438/3/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 94:         InsertOperator insertDeleteOp = (InsertOperator) op;
> name: insertOp might be better.
Done


Line 95:         assert insertDeleteOp.isBulkload();
> I know this has been sit here a long time. Other than an assert, how about 
Done


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

Line 93:         InsertOperator insertDeleteOp = (InsertOperator) op;
> insertOp?
Done


https://asterix-gerrit.ics.uci.edu/#/c/438/3/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java:

Line 73:                     new HashMap<AbstractLogicalOperator, Boolean>());
> What's the purpose of Hashmap here?
I changed the plan printer to avoid printing duplicate operators in the pipeline. This happened quite often when a replicate operator is used. The printer starts traversing plan top-down and at some point the pipeline splits and later rejoins under replicate operator. However the printer was not aware of the fact that the operator is already printed. This map is used to determine that fact. I guess a HashSet could be used instead, but I don't think their implementations in Java are that different.


https://asterix-gerrit.ics.uci.edu/#/c/438/3/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
File algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java:

Line 42:     private final IScalarEvaluatorFactory[] evalFactories;
> What's the purpose of this change?
That was automatic change by IDE. It's considered a good practice to make  variables final by default wherever it is possible. But the change itself is not motivated by anything


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 3
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: Yes

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 4:

I'm kind of thinking it is broken, I wonder how though.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 4:

This review should is abandoned in favor to https://asterix-gerrit.ics.uci.edu/#/c/462/ due to commit branch change

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 3
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

Posted by "Ildar Absalyamov (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/438

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................

ASTERIXDB-1109: Fixed deletion of records from open secondary index

- Separated InsertDeleteOperator into two distinct operators
- Added a rule, which reuses PK vars, retrieved from datascan\idxLookup
- Changed plan printer for insert\delete\idxInsert\idxDelete operators
- Modified plan printer to skip repeated parts of the plan under replicate operator

Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
---
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/metadata/IMetadataProvider.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractModificationOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/DeleteOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/IndexInsertDeleteOperator.java
A algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/InsertOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/TokenizeOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/BulkloadPOperator.java
C algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/DeletePOperator.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/IndexInsertDeletePOperator.java
R algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/InsertPOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/TokenizePOperator.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/PlanPrettyPrinter.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
M algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.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
M algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
M algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
32 files changed, 578 insertions(+), 377 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 3
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 4: -Verified

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 3:

(6 comments)

https://asterix-gerrit.ics.uci.edu/#/c/438/3/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java:

Line 32:     INDEX_INSERT_DELETE,
What's the main difference between index insert/delete and insert/delete? I'm asking this since iNDEX_INSERT_DELETE is one op and you have divided INSERT_DELETE into two ops.


https://asterix-gerrit.ics.uci.edu/#/c/438/3/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 94:         InsertOperator insertDeleteOp = (InsertOperator) op;
name: insertOp might be better.


Line 95:         assert insertDeleteOp.isBulkload();
I know this has been sit here a long time. Other than an assert, how about throwing an exception?


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

Line 93:         InsertOperator insertDeleteOp = (InsertOperator) op;
insertOp?


https://asterix-gerrit.ics.uci.edu/#/c/438/3/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java
File algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/AbstractRuleController.java:

Line 73:                     new HashMap<AbstractLogicalOperator, Boolean>());
What's the purpose of Hashmap here?


https://asterix-gerrit.ics.uci.edu/#/c/438/3/algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java
File algebricks/algebricks-runtime/src/main/java/org/apache/hyracks/algebricks/runtime/operators/std/AssignRuntimeFactory.java:

Line 42:     private final IScalarEvaluatorFactory[] evalFactories;
What's the purpose of this change?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 3
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: Yes

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 4:

Hmmmmm. What is going on here?.... It looks like the file channel in the IOManager is null?

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 4:

So what does that mean? Do I need to somehow change the branch?

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 4:

Ooh, wait, actually nevermind. git-gerrit doesn't have this yet. Have you ever submitted a squashed commit to gerrit by hand, then?

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 4:

No. Is it described somewhere?
Is the build broken again? Seems like it's not sporadic.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 1
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Abandoned

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

Gerrit-MessageType: abandon
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>

Change in hyracks[master]: ASTERIXDB-1109: Fixed deletion of records from open secondar...

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

Change subject: ASTERIXDB-1109: Fixed deletion of records from open secondary index
......................................................................


Patch Set 4:

Yeah, the feature was added to git-gerrit not so long ago: https://github.com/ceejatec/git-gerrit/commit/a31e270020df74601c124a62486918491628e16c

Basically the squashed commit needs to go to refs/for/release-0.8.8 instead of refs/for/master . It obviously should also be based on that branch.

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Ib2036d2eac4b0a0c6ac2c2e7e1bac383b11106bd
Gerrit-PatchSet: 4
Gerrit-Project: hyracks
Gerrit-Branch: master
Gerrit-Owner: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Ildar Absalyamov <il...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Young-Seok Kim <ki...@gmail.com>
Gerrit-HasComments: No