merge revision from Sattam
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/MaterializeOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/MaterializeOperator.java
new file mode 100644
index 0000000..a8e8049
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/MaterializeOperator.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.algebra.operators;
+
+import java.util.Collection;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractExtensibleLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorExtension;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+
+public class MaterializeOperator extends AbstractExtensibleLogicalOperator {
+
+    @Override
+    public boolean isMap() {
+        return false;
+    }
+
+    @Override
+    public IOperatorExtension newInstance() {
+        return new MaterializeOperator();
+    }
+
+    @Override
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform transform) throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public String toString() {
+        return "materialize";
+    }
+
+    @Override
+    public void getUsedVariables(Collection<LogicalVariable> usedVars) {
+        // No used variables.
+    }
+
+    @Override
+    public void getProducedVariables(Collection<LogicalVariable> producedVars) {
+        // No produced variables.
+    }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/MaterializePOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/MaterializePOperator.java
new file mode 100644
index 0000000..71948c9
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/MaterializePOperator.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.algebra.operators.physical;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractPhysicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.misc.MaterializingOperatorDescriptor;
+
+public class MaterializePOperator extends AbstractPhysicalOperator {
+
+    @Override
+    public PhysicalOperatorTag getOperatorTag() {
+        return PhysicalOperatorTag.EXTENSION_OPERATOR;
+    }
+
+    @Override
+    public String toString() {
+        return "MATERIALIZE";
+    }
+
+    @Override
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+            IPhysicalPropertiesVector reqdByParent) {
+        return emptyUnaryRequirements();
+    }
+
+    @Override
+    public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+        deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+    }
+
+    @Override
+    public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+            IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+            throws AlgebricksException {
+        RecordDescriptor recDescriptor = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op),
+                propagatedSchema, context);
+        MaterializingOperatorDescriptor materializationOpDesc = new MaterializingOperatorDescriptor(
+                builder.getJobSpec(), recDescriptor);
+        contributeOpDesc(builder, (AbstractLogicalOperator) op, materializationOpDesc);
+        ILogicalOperator src = op.getInputs().get(0).getValue();
+        builder.contributeGraphEdge(src, 0, op, 0);
+    }
+
+    @Override
+    public boolean isMicroOperator() {
+        return true;
+    }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
index 70306d6..70ed7ef 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
@@ -32,6 +32,7 @@
 import edu.uci.ics.asterix.optimizer.rules.IntroduceDynamicTypeCastRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceEnforcedListTypeRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceInstantLockSearchCallbackRule;
+import edu.uci.ics.asterix.optimizer.rules.IntroduceMaterializationForInsertWithSelfScanRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceRapidFrameFlushProjectRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceSecondaryIndexInsertDeleteRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceStaticTypeCastForInsertRule;
@@ -244,6 +245,7 @@
         physicalRewritesAllLevels.add(new PullPositionalVariableFromUnnestRule());
         physicalRewritesAllLevels.add(new PushProjectDownRule());
         physicalRewritesAllLevels.add(new InsertProjectBeforeUnionRule());
+        physicalRewritesAllLevels.add(new IntroduceMaterializationForInsertWithSelfScanRule());
         physicalRewritesAllLevels.add(new InlineSingleReferenceVariablesRule());
         physicalRewritesAllLevels.add(new RemoveUnusedAssignAndAggregateRule());
         physicalRewritesAllLevels.add(new ConsolidateAssignsRule());
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
new file mode 100644
index 0000000..e62e4db
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
@@ -0,0 +1,123 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.optimizer.rules;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.asterix.algebra.operators.MaterializeOperator;
+import edu.uci.ics.asterix.algebra.operators.physical.MaterializePOperator;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource.AqlDataSourceType;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.optimizer.rules.am.AccessMethodJobGenParams;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class IntroduceMaterializationForInsertWithSelfScanRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.INSERT_DELETE) {
+            return false;
+        }
+
+        InsertDeleteOperator insertOp = (InsertDeleteOperator) op;
+        boolean sameDataset = checkIfInsertAndScanDatasetsSame(op, ((AqlDataSource) insertOp.getDataSource())
+                .getDataset().getDatasetName());
+
+        if (sameDataset) {
+            MaterializeOperator materializeOperator = new MaterializeOperator();
+            MaterializePOperator materializePOperator = new MaterializePOperator();
+            materializeOperator.setPhysicalOperator(materializePOperator);
+
+            ExtensionOperator extensionOperator = new ExtensionOperator(materializeOperator);
+            extensionOperator.setPhysicalOperator(materializePOperator);
+
+            extensionOperator.getInputs().add(
+                    new MutableObject<ILogicalOperator>(insertOp.getInputs().get(0).getValue()));
+            //extensionOperator.setExecutionMode(ExecutionMode.LOCAL);
+            context.computeAndSetTypeEnvironmentForOperator(extensionOperator);
+
+            insertOp.getInputs().clear();
+            insertOp.getInputs().add(new MutableObject<ILogicalOperator>(extensionOperator));
+            context.computeAndSetTypeEnvironmentForOperator(insertOp);
+            return true;
+        } else {
+            return false;
+        }
+
+    }
+
+    private boolean checkIfInsertAndScanDatasetsSame(AbstractLogicalOperator op, String insertDatasetName) {
+        boolean sameDataset = false;
+        for (int i = 0; i < op.getInputs().size(); ++i) {
+            AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
+
+            if (descendantOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+                UnnestMapOperator unnestMapOp = (UnnestMapOperator) descendantOp;
+                ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
+                if (unnestExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
+                    FunctionIdentifier fid = f.getFunctionIdentifier();
+                    if (!fid.equals(AsterixBuiltinFunctions.INDEX_SEARCH)) {
+                        throw new IllegalStateException();
+                    }
+                    AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
+                    jobGenParams.readFromFuncArgs(f.getArguments());
+                    boolean isPrimaryIndex = jobGenParams.isPrimaryIndex();
+                    String indexName = jobGenParams.getIndexName();
+                    if (isPrimaryIndex && indexName.compareTo(insertDatasetName) == 0) {
+                        return true;
+                    }
+                }
+            } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+                DataSourceScanOperator dataSourceScanOp = (DataSourceScanOperator) descendantOp;
+                AqlDataSource ds = (AqlDataSource) dataSourceScanOp.getDataSource();
+                if (ds.getDatasourceType() != AqlDataSourceType.FEED
+                        && ds.getDatasourceType() != AqlDataSourceType.EXTERNAL_FEED) {
+                    if (ds.getDataset().getDatasetName().compareTo(insertDatasetName) == 0) {
+                        return true;
+                    }
+                }
+            }
+            sameDataset = checkIfInsertAndScanDatasetsSame(descendantOp, insertDatasetName);
+            if (sameDataset) {
+                break;
+            }
+        }
+        return sameDataset;
+    }
+}
diff --git a/asterix-app/data/odd-numbers-2.adm b/asterix-app/data/odd-numbers-2.adm
new file mode 100644
index 0000000..c079eeb
--- /dev/null
+++ b/asterix-app/data/odd-numbers-2.adm
@@ -0,0 +1,5 @@
+{"id":1}
+{"id":3}
+{"id":5}
+{"id":7}
+{"id":9}
\ No newline at end of file
diff --git a/asterix-app/data/odd-numbers.adm b/asterix-app/data/odd-numbers.adm
new file mode 100644
index 0000000..8f400b6
--- /dev/null
+++ b/asterix-app/data/odd-numbers.adm
@@ -0,0 +1,10 @@
+{"id":1}
+{"id":3}
+{"id":5}
+{"id":7}
+{"id":9}
+{"id":11}
+{"id":13}
+{"id":15}
+{"id":17}
+{"id":19}
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset-with-index.aql b/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset-with-index.aql
new file mode 100644
index 0000000..4643eb6
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset-with-index.aql
@@ -0,0 +1,33 @@
+/* 
+ * Test case Name  : insert-and-scan-dataset-with-index.aql
+ * Description     : This test is intended to test inserting into a dataset that has a secondary index and scan
+ * the data at the same time where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date            : July 11 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+create type test.Emp as closed {
+id:int32,
+fname:string,
+lname:string,
+age:int32,
+dept:string
+}
+
+create dataset test.employee(Emp) primary key id;
+
+create index idx_employee_first_name on test.employee(fname);
+
+insert into dataset test.employee (
+for $x in dataset test.employee
+return {
+	"id": $x.id + 10000,
+	"fname": $x.fname,
+	"lname": $x.lname,
+	"age": $x.age,
+	"dept": $x.dept
+}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.aql b/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.aql
new file mode 100644
index 0000000..52c4384
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-dataset.aql
@@ -0,0 +1,26 @@
+/* 
+ * Test case Name  : insert-and-scan-dataset.aql
+ * Description     : This test is intended to test inserting into a dataset and scan it at the same time
+ * where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date            : July 11 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type myDataType as open {
+  id: int32
+}
+
+create dataset myData(myDataType)
+  primary key id;
+
+insert into dataset myData (
+for $x in dataset myData
+return {
+	"id": $x.id + 1
+}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-joined-datasets.aql b/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-joined-datasets.aql
new file mode 100644
index 0000000..a8a27ca
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/insert-and-scan-joined-datasets.aql
@@ -0,0 +1,32 @@
+/* 
+ * Test case Name  : insert-and-scan-joined-datasets.aql
+ * Description     : This test is intended to test inserting into a dataset where the incoming stream
+  is involve a join operation that has the same dataset. We insert a materializing to prevent the 
+  possibility of deadlatch.
+ * Expected Result : Success
+ * Date            : July 11 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type myDataType as open {
+  id: int32
+}
+
+create dataset myData(myDataType)
+  primary key id;
+  
+create dataset myData2(myDataType)
+  primary key id;
+
+insert into dataset myData (
+for $x in dataset myData2
+for $y in dataset myData
+where $x.id = $y.id
+return {
+	"id": $x.id + 1
+}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset-with-index.plan b/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset-with-index.plan
new file mode 100644
index 0000000..8bc296b
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset-with-index.plan
@@ -0,0 +1,20 @@
+-- COMMIT  |PARTITIONED|
+  -- STREAM_PROJECT  |PARTITIONED|
+    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+      -- INDEX_INSERT_DELETE  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- INSERT_DELETE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- MATERIALIZE  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$10]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan b/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan
new file mode 100644
index 0000000..e11e2a8
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-dataset.plan
@@ -0,0 +1,15 @@
+-- COMMIT  |PARTITIONED|
+  -- STREAM_PROJECT  |PARTITIONED|
+    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+      -- INSERT_DELETE  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- MATERIALIZE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$6]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-joined-datasets.plan b/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-joined-datasets.plan
new file mode 100644
index 0000000..88f256e
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/insert-and-scan-joined-datasets.plan
@@ -0,0 +1,25 @@
+-- COMMIT  |PARTITIONED|
+  -- STREAM_PROJECT  |PARTITIONED|
+    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+      -- INSERT_DELETE  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- MATERIALIZE  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$11]  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$14][$$15]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ddl.aql
new file mode 100644
index 0000000..6ded7a7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.ddl.aql
@@ -0,0 +1,22 @@
+/* 
+ * Test case Name  : insert-and-scan-dataset-with-index.aql
+ * Description     : This test is intended to test inserting into a dataset that has a secondary index and scan
+ * the data at the same time where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date            : July 11 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+create type test.Emp as closed {
+id:int32,
+fname:string,
+lname:string,
+age:int32,
+dept:string
+}
+
+create dataset test.employee(Emp) primary key id;
+
+create index idx_employee_first_name on test.employee(fname);
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql
new file mode 100644
index 0000000..cf7b309
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql
@@ -0,0 +1,27 @@
+/* 
+ * Test case Name  : insert-and-scan-dataset-with-index.aql
+ * Description     : This test is intended to test inserting into a dataset that has a secondary index and scan
+ * the data at the same time where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date            : July 11 2013
+ */
+ 
+use dataverse test;
+
+use dataverse test;
+
+load dataset test.employee
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/names.adm"),("format"="delimited-text"),("delimiter"="|"));
+
+
+insert into dataset test.employee (
+for $x in dataset test.employee
+return {
+	"id": $x.id + 10000,
+	"fname": $x.fname,
+	"lname": $x.lname,
+	"age": $x.age,
+	"dept": $x.dept
+}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.query.aql
new file mode 100644
index 0000000..ae259a9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.3.query.aql
@@ -0,0 +1,13 @@
+/* 
+ * Test case Name  : insert-and-scan-dataset-with-index.aql
+ * Description     : This test is intended to test inserting into a dataset that has a secondary index and scan
+ * the data at the same time where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date            : July 11 2013
+ */
+ 
+use dataverse test;
+
+for $l in dataset('test.employee')
+order by $l.id
+return $l
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.1.ddl.aql
new file mode 100644
index 0000000..fc04212
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.1.ddl.aql
@@ -0,0 +1,19 @@
+/* 
+ * Test case Name  : insert-and-scan-dataset.aql
+ * Description     : This test is intended to test inserting into a dataset and scan it at the same time
+ * where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date            : July 11 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type myDataType as open {
+  id: int32
+}
+
+create dataset myData(myDataType)
+  primary key id;
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.2.update.aql
new file mode 100644
index 0000000..58e0c38
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.2.update.aql
@@ -0,0 +1,21 @@
+/* 
+ * Test case Name  : insert-and-scan-dataset.aql
+ * Description     : This test is intended to test inserting into a dataset and scan it at the same time
+ * where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date            : July 11 2013
+ */
+ 
+use dataverse test;
+
+load dataset myData 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/odd-numbers.adm"),("format"="adm"))pre-sorted;
+
+
+insert into dataset myData (
+for $x in dataset myData
+return {
+	"id": $x.id + 1
+}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.3.query.aql
new file mode 100644
index 0000000..f9af922
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset/insert-and-scan-dataset.3.query.aql
@@ -0,0 +1,13 @@
+/* 
+ * Test case Name  : insert-and-scan-dataset.aql
+ * Description     : This test is intended to test inserting into a dataset and scan it at the same time
+ * where we insert a materializing to prevent the possibility of deadlatch.
+ * Expected Result : Success
+ * Date            : July 11 2013
+ */
+ 
+use dataverse test;
+
+for $c in dataset('myData')
+order by $c.id
+return $c 
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.1.ddl.aql
new file mode 100644
index 0000000..adf6239
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.1.ddl.aql
@@ -0,0 +1,23 @@
+/* 
+ * Test case Name  : insert-and-scan-joined-datasets.aql
+ * Description     : This test is intended to test inserting into a dataset where the incoming stream
+  is involve a join operation that has the same dataset. We insert a materializing to prevent the 
+  possibility of deadlatch.
+ * Expected Result : Success
+ * Date            : July 11 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type myDataType as open {
+  id: int32
+}
+
+create dataset myData(myDataType)
+  primary key id;
+  
+create dataset myData2(myDataType)
+  primary key id;
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.2.update.aql
new file mode 100644
index 0000000..6b9e1b6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.2.update.aql
@@ -0,0 +1,27 @@
+/* 
+ * Test case Name  : insert-and-scan-joined-datasets.aql
+ * Description     : This test is intended to test inserting into a dataset where the incoming stream
+  is involve a join operation that has the same dataset. We insert a materializing to prevent the 
+  possibility of deadlatch.
+ * Expected Result : Success
+ * Date            : July 11 2013
+ */
+ 
+use dataverse test;
+
+load dataset myData 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/odd-numbers.adm"),("format"="adm"))pre-sorted;
+
+load dataset myData2 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/odd-numbers-2.adm"),("format"="adm"))pre-sorted;
+
+insert into dataset myData (
+for $x in dataset myData2
+for $y in dataset myData
+where $x.id = $y.id
+return {
+	"id": $x.id + 1
+}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.3.query.aql
new file mode 100644
index 0000000..3614e1b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.3.query.aql
@@ -0,0 +1,14 @@
+/* 
+ * Test case Name  : insert-and-scan-joined-datasets.aql
+ * Description     : This test is intended to test inserting into a dataset where the incoming stream
+  is involve a join operation that has the same dataset. We insert a materializing to prevent the 
+  possibility of deadlatch.
+ * Expected Result : Success
+ * Date            : July 11 2013
+ */
+ 
+use dataverse test;
+
+for $c in dataset('myData')
+order by $c.id
+return $c 
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.adm
new file mode 100644
index 0000000..bd9b76a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.1.adm
@@ -0,0 +1,240 @@
+{ "id": 101, "fname": "Javier", "lname": "Makuch", "age": 28, "dept": "IT" }
+{ "id": 110, "fname": "Allan", "lname": "Piland", "age": 29, "dept": "HR" }
+{ "id": 112, "fname": "Pearlie", "lname": "Aumann", "age": 31, "dept": "Payroll" }
+{ "id": 113, "fname": "Chandra", "lname": "Hase", "age": 34, "dept": "Sales" }
+{ "id": 114, "fname": "Christian", "lname": "Convery", "age": 28, "dept": "HR" }
+{ "id": 115, "fname": "Panther", "lname": "Ritch", "age": 26, "dept": "IT" }
+{ "id": 116, "fname": "Ted", "lname": "Elsea", "age": 26, "dept": "IT" }
+{ "id": 117, "fname": "Tabatha", "lname": "Bladen", "age": 25, "dept": "HR" }
+{ "id": 118, "fname": "Clayton", "lname": "Oltman", "age": 42, "dept": "Sales" }
+{ "id": 119, "fname": "Sharron", "lname": "Darwin", "age": 32, "dept": "Payroll" }
+{ "id": 210, "fname": "Clayton", "lname": "Durgin", "age": 52, "dept": "HR" }
+{ "id": 212, "fname": "Emilia", "lname": "Chenail", "age": 26, "dept": "Sales" }
+{ "id": 213, "fname": "Kenya", "lname": "Almquist", "age": 43, "dept": "Payroll" }
+{ "id": 214, "fname": "Alejandra", "lname": "Lacefield", "age": 41, "dept": "HR" }
+{ "id": 215, "fname": "Karina", "lname": "Michelsen", "age": 46, "dept": "IT" }
+{ "id": 216, "fname": "Katy", "lname": "Delillo", "age": 36, "dept": "IT" }
+{ "id": 217, "fname": "Benita", "lname": "Kleist", "age": 37, "dept": "HR" }
+{ "id": 218, "fname": "Earlene", "lname": "Paluch", "age": 31, "dept": "IT" }
+{ "id": 219, "fname": "Kurt", "lname": "Petermann", "age": 27, "dept": "Payroll" }
+{ "id": 299, "fname": "Julio", "lname": "Iorio", "age": 37, "dept": "IT" }
+{ "id": 363, "fname": "Cody", "lname": "Rodreguez", "age": 26, "dept": "IT" }
+{ "id": 404, "fname": "Emilia", "lname": "Square", "age": 32, "dept": "IT" }
+{ "id": 414, "fname": "Mathew", "lname": "Fuschetto", "age": 34, "dept": "HR" }
+{ "id": 424, "fname": "Allyson", "lname": "Remus", "age": 32, "dept": "IT" }
+{ "id": 434, "fname": "Earlene", "lname": "Linebarger", "age": 26, "dept": "Payroll" }
+{ "id": 444, "fname": "Clinton", "lname": "Sick", "age": 29, "dept": "IT" }
+{ "id": 454, "fname": "Ted", "lname": "Caba", "age": 28, "dept": "HR" }
+{ "id": 463, "fname": "Marcie", "lname": "States", "age": 28, "dept": "IT" }
+{ "id": 464, "fname": "Fernando", "lname": "Engelke", "age": 39, "dept": "IT" }
+{ "id": 474, "fname": "Mathew", "lname": "Courchesne", "age": 31, "dept": "IT" }
+{ "id": 484, "fname": "Cody", "lname": "Vinyard", "age": 36, "dept": "Payroll" }
+{ "id": 494, "fname": "Benita", "lname": "Fravel", "age": 33, "dept": "Sales" }
+{ "id": 504, "fname": "Erik", "lname": "Dobek", "age": 29, "dept": "IT" }
+{ "id": 514, "fname": "Julio", "lname": "Ruben", "age": 41, "dept": "IT" }
+{ "id": 524, "fname": "Benita", "lname": "Maltos", "age": 33, "dept": "IT" }
+{ "id": 534, "fname": "Kurt", "lname": "Biscoe", "age": 36, "dept": "HR" }
+{ "id": 538, "fname": "Milagros", "lname": "Forkey", "age": 34, "dept": "Sales" }
+{ "id": 544, "fname": "Loraine", "lname": "Housel", "age": 30, "dept": "Sales" }
+{ "id": 554, "fname": "Jamie", "lname": "Rachal", "age": 30, "dept": "IT" }
+{ "id": 564, "fname": "Liza", "lname": "Fredenburg", "age": 37, "dept": "IT" }
+{ "id": 574, "fname": "Ericka", "lname": "Feldmann", "age": 29, "dept": "Sales" }
+{ "id": 584, "fname": "Dollie", "lname": "Dattilo", "age": 32, "dept": "Payroll" }
+{ "id": 589, "fname": "Lorrie", "lname": "Sharon", "age": 27, "dept": "IT" }
+{ "id": 594, "fname": "Roxie", "lname": "Houghtaling", "age": 40, "dept": "Payroll" }
+{ "id": 601, "fname": "Neil", "lname": "Deforge", "age": 26, "dept": "HR" }
+{ "id": 611, "fname": "Earlene", "lname": "Marcy", "age": 32, "dept": "IT" }
+{ "id": 621, "fname": "Erik", "lname": "Lechuga", "age": 42, "dept": "Payroll" }
+{ "id": 631, "fname": "Tyrone", "lname": "Holtzclaw", "age": 34, "dept": "Sales" }
+{ "id": 641, "fname": "Lance", "lname": "Hankey", "age": 35, "dept": "Sales" }
+{ "id": 651, "fname": "Mallory", "lname": "Gladding", "age": 31, "dept": "HR" }
+{ "id": 661, "fname": "Tia", "lname": "Braaten", "age": 40, "dept": "IT" }
+{ "id": 671, "fname": "Julio", "lname": "Vanpatten", "age": 30, "dept": "Payroll" }
+{ "id": 681, "fname": "Max", "lname": "Teachout", "age": 34, "dept": "IT" }
+{ "id": 691, "fname": "Karina", "lname": "Wingerter", "age": 31, "dept": "IT" }
+{ "id": 711, "fname": "Hugh", "lname": "Lema", "age": 25, "dept": "HR" }
+{ "id": 721, "fname": "Schwan", "lname": "Phil", "age": 34, "dept": "Payroll" }
+{ "id": 732, "fname": "Noemi", "lname": "Eacret", "age": 56, "dept": "HR" }
+{ "id": 741, "fname": "Julio", "lname": "Mattocks", "age": 38, "dept": "Sales" }
+{ "id": 751, "fname": "Lance", "lname": "Kottke", "age": 34, "dept": "IT" }
+{ "id": 761, "fname": "Kurt", "lname": "Liz", "age": 32, "dept": "HR" }
+{ "id": 771, "fname": "Neva", "lname": "Barbeau", "age": 45, "dept": "Sales" }
+{ "id": 781, "fname": "Karina", "lname": "Tuthill", "age": 46, "dept": "Payroll" }
+{ "id": 791, "fname": "Maricela", "lname": "Cambron", "age": 36, "dept": "IT" }
+{ "id": 809, "fname": "Clayton", "lname": "Delany", "age": 23, "dept": "IT" }
+{ "id": 811, "fname": "Kubik", "lname": "Kuhn", "age": 27, "dept": "HR" }
+{ "id": 821, "fname": "Allan", "lname": "Tomes", "age": 29, "dept": "Payroll" }
+{ "id": 831, "fname": "Lonnie", "lname": "Aller", "age": 33, "dept": "Sales" }
+{ "id": 841, "fname": "Neil", "lname": "Hurrell", "age": 26, "dept": "IT" }
+{ "id": 851, "fname": "Clayton", "lname": "Engles", "age": 41, "dept": "HR" }
+{ "id": 861, "fname": "Javier", "lname": "Gabrielson", "age": 39, "dept": "Payroll" }
+{ "id": 871, "fname": "Allan", "lname": "Alejandre", "age": 48, "dept": "IT" }
+{ "id": 881, "fname": "Julio", "lname": "Isa", "age": 38, "dept": "Sales" }
+{ "id": 891, "fname": "Roslyn", "lname": "Simmerman", "age": 31, "dept": "IT" }
+{ "id": 915, "fname": "Starner", "lname": "Stuart", "age": 25, "dept": "Sales" }
+{ "id": 925, "fname": "Sofia", "lname": "Cuff", "age": 30, "dept": "HR" }
+{ "id": 935, "fname": "Milagros", "lname": "Murguia", "age": 31, "dept": "IT" }
+{ "id": 945, "fname": "Margery", "lname": "Haldeman", "age": 32, "dept": "IT" }
+{ "id": 955, "fname": "Max", "lname": "Mell", "age": 33, "dept": "HR" }
+{ "id": 965, "fname": "Micco", "lname": "Mercy", "age": 31, "dept": "Payroll" }
+{ "id": 975, "fname": "Clare", "lname": "Vangieson", "age": 34, "dept": "IT" }
+{ "id": 985, "fname": "Elnora", "lname": "Dimauro", "age": 35, "dept": "Sales" }
+{ "id": 995, "fname": "Pearlie", "lname": "Kocian", "age": 38, "dept": "HR" }
+{ "id": 1007, "fname": "Yingyi", "lname": "Bu", "age": 27, "dept": "IT" }
+{ "id": 1263, "fname": "Tania", "lname": "Loffredo", "age": 25, "dept": "IT" }
+{ "id": 1410, "fname": "Clinton", "lname": "Fredricks", "age": 34, "dept": "IT" }
+{ "id": 1411, "fname": "Lance", "lname": "Farquhar", "age": 32, "dept": "HR" }
+{ "id": 1412, "fname": "Tabatha", "lname": "Crisler", "age": 33, "dept": "IT" }
+{ "id": 1413, "fname": "Max", "lname": "Durney", "age": 29, "dept": "IT" }
+{ "id": 1414, "fname": "Carmella", "lname": "Strauser", "age": 30, "dept": "Payroll" }
+{ "id": 1415, "fname": "Kelly", "lname": "Carrales", "age": 40, "dept": "IT" }
+{ "id": 1416, "fname": "Guy", "lname": "Merten", "age": 29, "dept": "Sales" }
+{ "id": 1417, "fname": "Noreen", "lname": "Ruhland", "age": 29, "dept": "IT" }
+{ "id": 1418, "fname": "Julio", "lname": "Damore", "age": 27, "dept": "Sales" }
+{ "id": 1419, "fname": "Selena", "lname": "Truby", "age": 25, "dept": "HR" }
+{ "id": 1420, "fname": "Alejandra", "lname": "Commons", "age": 30, "dept": "Sales" }
+{ "id": 1421, "fname": "Allyson", "lname": "Balk", "age": 30, "dept": "IT" }
+{ "id": 1422, "fname": "Nelson", "lname": "Byun", "age": 40, "dept": "Sales" }
+{ "id": 1423, "fname": "Christian", "lname": "Reidhead", "age": 40, "dept": "IT" }
+{ "id": 1424, "fname": "Pearlie", "lname": "Hopkin", "age": 48, "dept": "Payroll" }
+{ "id": 1425, "fname": "Nelson", "lname": "Wohlers", "age": 41, "dept": "HR" }
+{ "id": 1426, "fname": "Marcie", "lname": "Rasnake", "age": 42, "dept": "Sales" }
+{ "id": 1427, "fname": "Hugh", "lname": "Marshburn", "age": 43, "dept": "Payroll" }
+{ "id": 1428, "fname": "Mathew", "lname": "Marasco", "age": 45, "dept": "Sales" }
+{ "id": 1429, "fname": "Kurt", "lname": "Veres", "age": 32, "dept": "IT" }
+{ "id": 1430, "fname": "Julio", "lname": "Barkett", "age": 39, "dept": "Sales" }
+{ "id": 1863, "fname": "Darren", "lname": "Thorington", "age": 32, "dept": "Sales" }
+{ "id": 1999, "fname": "Susan", "lname": "Malaika", "age": 42, "dept": "HR" }
+{ "id": 2333, "fname": "Chen", "lname": "Li", "age": 42, "dept": "HR" }
+{ "id": 2963, "fname": "Neil", "lname": "Gunnerson", "age": 34, "dept": "IT" }
+{ "id": 3563, "fname": "Hazeltine", "lname": "Susan", "age": 29, "dept": "Sales" }
+{ "id": 3666, "fname": "Young Seok", "lname": "Kim", "age": 35, "dept": "Payroll" }
+{ "id": 4727, "fname": "Michael", "lname": "Carey", "age": 50, "dept": "Payroll" }
+{ "id": 5438, "fname": "Lakisha", "lname": "Quashie", "age": 29, "dept": "HR" }
+{ "id": 7444, "fname": "Sharad", "lname": "Mehrotra", "age": 42, "dept": "Sales" }
+{ "id": 7663, "fname": "Annabelle", "lname": "Nimmo", "age": 30, "dept": "Payroll" }
+{ "id": 8301, "fname": "Earlene", "lname": "Wallick", "age": 26, "dept": "HR" }
+{ "id": 8338, "fname": "Julio", "lname": "Bosket", "age": 28, "dept": "Payroll" }
+{ "id": 9555, "fname": "Tony", "lname": "Givargis", "age": 40, "dept": "Sales" }
+{ "id": 9763, "fname": "Ted", "lname": "Saini", "age": 31, "dept": "IT" }
+{ "id": 9941, "fname": "Khurram Faraaz", "lname": "Mohammed", "age": 30, "dept": "HR" }
+{ "id": 10101, "fname": "Javier", "lname": "Makuch", "age": 28, "dept": "IT" }
+{ "id": 10110, "fname": "Allan", "lname": "Piland", "age": 29, "dept": "HR" }
+{ "id": 10112, "fname": "Pearlie", "lname": "Aumann", "age": 31, "dept": "Payroll" }
+{ "id": 10113, "fname": "Chandra", "lname": "Hase", "age": 34, "dept": "Sales" }
+{ "id": 10114, "fname": "Christian", "lname": "Convery", "age": 28, "dept": "HR" }
+{ "id": 10115, "fname": "Panther", "lname": "Ritch", "age": 26, "dept": "IT" }
+{ "id": 10116, "fname": "Ted", "lname": "Elsea", "age": 26, "dept": "IT" }
+{ "id": 10117, "fname": "Tabatha", "lname": "Bladen", "age": 25, "dept": "HR" }
+{ "id": 10118, "fname": "Clayton", "lname": "Oltman", "age": 42, "dept": "Sales" }
+{ "id": 10119, "fname": "Sharron", "lname": "Darwin", "age": 32, "dept": "Payroll" }
+{ "id": 10210, "fname": "Clayton", "lname": "Durgin", "age": 52, "dept": "HR" }
+{ "id": 10212, "fname": "Emilia", "lname": "Chenail", "age": 26, "dept": "Sales" }
+{ "id": 10213, "fname": "Kenya", "lname": "Almquist", "age": 43, "dept": "Payroll" }
+{ "id": 10214, "fname": "Alejandra", "lname": "Lacefield", "age": 41, "dept": "HR" }
+{ "id": 10215, "fname": "Karina", "lname": "Michelsen", "age": 46, "dept": "IT" }
+{ "id": 10216, "fname": "Katy", "lname": "Delillo", "age": 36, "dept": "IT" }
+{ "id": 10217, "fname": "Benita", "lname": "Kleist", "age": 37, "dept": "HR" }
+{ "id": 10218, "fname": "Earlene", "lname": "Paluch", "age": 31, "dept": "IT" }
+{ "id": 10219, "fname": "Kurt", "lname": "Petermann", "age": 27, "dept": "Payroll" }
+{ "id": 10299, "fname": "Julio", "lname": "Iorio", "age": 37, "dept": "IT" }
+{ "id": 10363, "fname": "Cody", "lname": "Rodreguez", "age": 26, "dept": "IT" }
+{ "id": 10404, "fname": "Emilia", "lname": "Square", "age": 32, "dept": "IT" }
+{ "id": 10414, "fname": "Mathew", "lname": "Fuschetto", "age": 34, "dept": "HR" }
+{ "id": 10424, "fname": "Allyson", "lname": "Remus", "age": 32, "dept": "IT" }
+{ "id": 10434, "fname": "Earlene", "lname": "Linebarger", "age": 26, "dept": "Payroll" }
+{ "id": 10444, "fname": "Clinton", "lname": "Sick", "age": 29, "dept": "IT" }
+{ "id": 10454, "fname": "Ted", "lname": "Caba", "age": 28, "dept": "HR" }
+{ "id": 10463, "fname": "Marcie", "lname": "States", "age": 28, "dept": "IT" }
+{ "id": 10464, "fname": "Fernando", "lname": "Engelke", "age": 39, "dept": "IT" }
+{ "id": 10474, "fname": "Mathew", "lname": "Courchesne", "age": 31, "dept": "IT" }
+{ "id": 10484, "fname": "Cody", "lname": "Vinyard", "age": 36, "dept": "Payroll" }
+{ "id": 10494, "fname": "Benita", "lname": "Fravel", "age": 33, "dept": "Sales" }
+{ "id": 10504, "fname": "Erik", "lname": "Dobek", "age": 29, "dept": "IT" }
+{ "id": 10514, "fname": "Julio", "lname": "Ruben", "age": 41, "dept": "IT" }
+{ "id": 10524, "fname": "Benita", "lname": "Maltos", "age": 33, "dept": "IT" }
+{ "id": 10534, "fname": "Kurt", "lname": "Biscoe", "age": 36, "dept": "HR" }
+{ "id": 10538, "fname": "Milagros", "lname": "Forkey", "age": 34, "dept": "Sales" }
+{ "id": 10544, "fname": "Loraine", "lname": "Housel", "age": 30, "dept": "Sales" }
+{ "id": 10554, "fname": "Jamie", "lname": "Rachal", "age": 30, "dept": "IT" }
+{ "id": 10564, "fname": "Liza", "lname": "Fredenburg", "age": 37, "dept": "IT" }
+{ "id": 10574, "fname": "Ericka", "lname": "Feldmann", "age": 29, "dept": "Sales" }
+{ "id": 10584, "fname": "Dollie", "lname": "Dattilo", "age": 32, "dept": "Payroll" }
+{ "id": 10589, "fname": "Lorrie", "lname": "Sharon", "age": 27, "dept": "IT" }
+{ "id": 10594, "fname": "Roxie", "lname": "Houghtaling", "age": 40, "dept": "Payroll" }
+{ "id": 10601, "fname": "Neil", "lname": "Deforge", "age": 26, "dept": "HR" }
+{ "id": 10611, "fname": "Earlene", "lname": "Marcy", "age": 32, "dept": "IT" }
+{ "id": 10621, "fname": "Erik", "lname": "Lechuga", "age": 42, "dept": "Payroll" }
+{ "id": 10631, "fname": "Tyrone", "lname": "Holtzclaw", "age": 34, "dept": "Sales" }
+{ "id": 10641, "fname": "Lance", "lname": "Hankey", "age": 35, "dept": "Sales" }
+{ "id": 10651, "fname": "Mallory", "lname": "Gladding", "age": 31, "dept": "HR" }
+{ "id": 10661, "fname": "Tia", "lname": "Braaten", "age": 40, "dept": "IT" }
+{ "id": 10671, "fname": "Julio", "lname": "Vanpatten", "age": 30, "dept": "Payroll" }
+{ "id": 10681, "fname": "Max", "lname": "Teachout", "age": 34, "dept": "IT" }
+{ "id": 10691, "fname": "Karina", "lname": "Wingerter", "age": 31, "dept": "IT" }
+{ "id": 10711, "fname": "Hugh", "lname": "Lema", "age": 25, "dept": "HR" }
+{ "id": 10721, "fname": "Schwan", "lname": "Phil", "age": 34, "dept": "Payroll" }
+{ "id": 10732, "fname": "Noemi", "lname": "Eacret", "age": 56, "dept": "HR" }
+{ "id": 10741, "fname": "Julio", "lname": "Mattocks", "age": 38, "dept": "Sales" }
+{ "id": 10751, "fname": "Lance", "lname": "Kottke", "age": 34, "dept": "IT" }
+{ "id": 10761, "fname": "Kurt", "lname": "Liz", "age": 32, "dept": "HR" }
+{ "id": 10771, "fname": "Neva", "lname": "Barbeau", "age": 45, "dept": "Sales" }
+{ "id": 10781, "fname": "Karina", "lname": "Tuthill", "age": 46, "dept": "Payroll" }
+{ "id": 10791, "fname": "Maricela", "lname": "Cambron", "age": 36, "dept": "IT" }
+{ "id": 10809, "fname": "Clayton", "lname": "Delany", "age": 23, "dept": "IT" }
+{ "id": 10811, "fname": "Kubik", "lname": "Kuhn", "age": 27, "dept": "HR" }
+{ "id": 10821, "fname": "Allan", "lname": "Tomes", "age": 29, "dept": "Payroll" }
+{ "id": 10831, "fname": "Lonnie", "lname": "Aller", "age": 33, "dept": "Sales" }
+{ "id": 10841, "fname": "Neil", "lname": "Hurrell", "age": 26, "dept": "IT" }
+{ "id": 10851, "fname": "Clayton", "lname": "Engles", "age": 41, "dept": "HR" }
+{ "id": 10861, "fname": "Javier", "lname": "Gabrielson", "age": 39, "dept": "Payroll" }
+{ "id": 10871, "fname": "Allan", "lname": "Alejandre", "age": 48, "dept": "IT" }
+{ "id": 10881, "fname": "Julio", "lname": "Isa", "age": 38, "dept": "Sales" }
+{ "id": 10891, "fname": "Roslyn", "lname": "Simmerman", "age": 31, "dept": "IT" }
+{ "id": 10915, "fname": "Starner", "lname": "Stuart", "age": 25, "dept": "Sales" }
+{ "id": 10925, "fname": "Sofia", "lname": "Cuff", "age": 30, "dept": "HR" }
+{ "id": 10935, "fname": "Milagros", "lname": "Murguia", "age": 31, "dept": "IT" }
+{ "id": 10945, "fname": "Margery", "lname": "Haldeman", "age": 32, "dept": "IT" }
+{ "id": 10955, "fname": "Max", "lname": "Mell", "age": 33, "dept": "HR" }
+{ "id": 10965, "fname": "Micco", "lname": "Mercy", "age": 31, "dept": "Payroll" }
+{ "id": 10975, "fname": "Clare", "lname": "Vangieson", "age": 34, "dept": "IT" }
+{ "id": 10985, "fname": "Elnora", "lname": "Dimauro", "age": 35, "dept": "Sales" }
+{ "id": 10995, "fname": "Pearlie", "lname": "Kocian", "age": 38, "dept": "HR" }
+{ "id": 11007, "fname": "Yingyi", "lname": "Bu", "age": 27, "dept": "IT" }
+{ "id": 11263, "fname": "Tania", "lname": "Loffredo", "age": 25, "dept": "IT" }
+{ "id": 11410, "fname": "Clinton", "lname": "Fredricks", "age": 34, "dept": "IT" }
+{ "id": 11411, "fname": "Lance", "lname": "Farquhar", "age": 32, "dept": "HR" }
+{ "id": 11412, "fname": "Tabatha", "lname": "Crisler", "age": 33, "dept": "IT" }
+{ "id": 11413, "fname": "Max", "lname": "Durney", "age": 29, "dept": "IT" }
+{ "id": 11414, "fname": "Carmella", "lname": "Strauser", "age": 30, "dept": "Payroll" }
+{ "id": 11415, "fname": "Kelly", "lname": "Carrales", "age": 40, "dept": "IT" }
+{ "id": 11416, "fname": "Guy", "lname": "Merten", "age": 29, "dept": "Sales" }
+{ "id": 11417, "fname": "Noreen", "lname": "Ruhland", "age": 29, "dept": "IT" }
+{ "id": 11418, "fname": "Julio", "lname": "Damore", "age": 27, "dept": "Sales" }
+{ "id": 11419, "fname": "Selena", "lname": "Truby", "age": 25, "dept": "HR" }
+{ "id": 11420, "fname": "Alejandra", "lname": "Commons", "age": 30, "dept": "Sales" }
+{ "id": 11421, "fname": "Allyson", "lname": "Balk", "age": 30, "dept": "IT" }
+{ "id": 11422, "fname": "Nelson", "lname": "Byun", "age": 40, "dept": "Sales" }
+{ "id": 11423, "fname": "Christian", "lname": "Reidhead", "age": 40, "dept": "IT" }
+{ "id": 11424, "fname": "Pearlie", "lname": "Hopkin", "age": 48, "dept": "Payroll" }
+{ "id": 11425, "fname": "Nelson", "lname": "Wohlers", "age": 41, "dept": "HR" }
+{ "id": 11426, "fname": "Marcie", "lname": "Rasnake", "age": 42, "dept": "Sales" }
+{ "id": 11427, "fname": "Hugh", "lname": "Marshburn", "age": 43, "dept": "Payroll" }
+{ "id": 11428, "fname": "Mathew", "lname": "Marasco", "age": 45, "dept": "Sales" }
+{ "id": 11429, "fname": "Kurt", "lname": "Veres", "age": 32, "dept": "IT" }
+{ "id": 11430, "fname": "Julio", "lname": "Barkett", "age": 39, "dept": "Sales" }
+{ "id": 11863, "fname": "Darren", "lname": "Thorington", "age": 32, "dept": "Sales" }
+{ "id": 11999, "fname": "Susan", "lname": "Malaika", "age": 42, "dept": "HR" }
+{ "id": 12333, "fname": "Chen", "lname": "Li", "age": 42, "dept": "HR" }
+{ "id": 12963, "fname": "Neil", "lname": "Gunnerson", "age": 34, "dept": "IT" }
+{ "id": 13563, "fname": "Hazeltine", "lname": "Susan", "age": 29, "dept": "Sales" }
+{ "id": 13666, "fname": "Young Seok", "lname": "Kim", "age": 35, "dept": "Payroll" }
+{ "id": 14727, "fname": "Michael", "lname": "Carey", "age": 50, "dept": "Payroll" }
+{ "id": 15438, "fname": "Lakisha", "lname": "Quashie", "age": 29, "dept": "HR" }
+{ "id": 17444, "fname": "Sharad", "lname": "Mehrotra", "age": 42, "dept": "Sales" }
+{ "id": 17663, "fname": "Annabelle", "lname": "Nimmo", "age": 30, "dept": "Payroll" }
+{ "id": 18301, "fname": "Earlene", "lname": "Wallick", "age": 26, "dept": "HR" }
+{ "id": 18338, "fname": "Julio", "lname": "Bosket", "age": 28, "dept": "Payroll" }
+{ "id": 19555, "fname": "Tony", "lname": "Givargis", "age": 40, "dept": "Sales" }
+{ "id": 19763, "fname": "Ted", "lname": "Saini", "age": 31, "dept": "IT" }
+{ "id": 19941, "fname": "Khurram Faraaz", "lname": "Mohammed", "age": 30, "dept": "HR" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-dataset/insert-and-scan-dataset.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-dataset/insert-and-scan-dataset.1.adm
new file mode 100644
index 0000000..29267a9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-dataset/insert-and-scan-dataset.1.adm
@@ -0,0 +1,20 @@
+{ "id": 1 }
+{ "id": 2 }
+{ "id": 3 }
+{ "id": 4 }
+{ "id": 5 }
+{ "id": 6 }
+{ "id": 7 }
+{ "id": 8 }
+{ "id": 9 }
+{ "id": 10 }
+{ "id": 11 }
+{ "id": 12 }
+{ "id": 13 }
+{ "id": 14 }
+{ "id": 15 }
+{ "id": 16 }
+{ "id": 17 }
+{ "id": 18 }
+{ "id": 19 }
+{ "id": 20 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.1.adm
new file mode 100644
index 0000000..73a793c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/insert-and-scan-joined-datasets/insert-and-scan-joined-datasets.1.adm
@@ -0,0 +1,15 @@
+{ "id": 1 }
+{ "id": 2 }
+{ "id": 3 }
+{ "id": 4 }
+{ "id": 5 }
+{ "id": 6 }
+{ "id": 7 }
+{ "id": 8 }
+{ "id": 9 }
+{ "id": 10 }
+{ "id": 11 }
+{ "id": 13 }
+{ "id": 15 }
+{ "id": 17 }
+{ "id": 19 }
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index 1e65071..5c56084 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -965,8 +965,8 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
-      <compilation-unit name="insert-into-empty-dataset-with-index">
-        <output-dir compare="Text">insert-into-empty-dataset-with-index</output-dir>
+      <compilation-unit name="insert-into-empty-dataset">
+        <output-dir compare="Text">insert-into-empty-dataset</output-dir>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
@@ -980,6 +980,21 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
+      <compilation-unit name="insert-and-scan-dataset">
+        <output-dir compare="Text">insert-and-scan-dataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-and-scan-dataset-with-index">
+        <output-dir compare="Text">insert-and-scan-dataset-with-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-and-scan-joined-datasets">
+        <output-dir compare="Text">insert-and-scan-joined-datasets</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
       <compilation-unit name="insert-into-loaded-dataset-with-index_01">
         <output-dir compare="Text">insert-into-loaded-dataset-with-index_01</output-dir>
       </compilation-unit>