Merge branch 'master' into eugenia/black_cherry_stable
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..dc25ae7
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/MaterializePOperator.java
@@ -0,0 +1,82 @@
+/*
+ * 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 {
+
+ private final boolean isSingleActivity;
+
+ public MaterializePOperator(boolean isSingleActivity) {
+ this.isSingleActivity = isSingleActivity;
+ }
+
+ @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, isSingleActivity);
+ 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 ab3516a..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());
@@ -271,6 +273,7 @@
// Re-infer all types, so that, e.g., the effect of not-is-null is
// propagated.
prepareForJobGenRewrites.add(new ReinferAllTypesRule());
+ prepareForJobGenRewrites.add(new SetExecutionModeRule());
return prepareForJobGenRewrites;
}
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..c15b0f0
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceMaterializationForInsertWithSelfScanRule.java
@@ -0,0 +1,121 @@
+/*
+ * 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.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(true);
+ materializeOperator.setPhysicalOperator(materializePOperator);
+
+ ExtensionOperator extensionOperator = new ExtensionOperator(materializeOperator);
+ extensionOperator.setPhysicalOperator(materializePOperator);
+
+ extensionOperator.getInputs().add(
+ new MutableObject<ILogicalOperator>(insertOp.getInputs().get(0).getValue()));
+ 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/noncollocated.aql b/asterix-app/src/test/resources/optimizerts/queries/collocated.aql
similarity index 78%
rename from asterix-app/src/test/resources/optimizerts/queries/noncollocated.aql
rename to asterix-app/src/test/resources/optimizerts/queries/collocated.aql
index 26b5ab0..bcd0a0d 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/noncollocated.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/collocated.aql
@@ -16,15 +16,11 @@
lottery_numbers: {{int32}}
}
-create nodegroup group1 if not exists on nc1, nc2;
-
-create nodegroup group2 if not exists on nc2;
-
create dataset Users(UserType)
- primary key uid on group1;
+ primary key uid;
create dataset Visitors(VisitorType)
- primary key vid on group2;
+ primary key vid;
write output to nc1:"/tmp/fuzzy1.adm";
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/noncollocated.plan b/asterix-app/src/test/resources/optimizerts/results/collocated.plan
similarity index 88%
rename from asterix-app/src/test/resources/optimizerts/results/noncollocated.plan
rename to asterix-app/src/test/resources/optimizerts/results/collocated.plan
index e92a84c..116ff8a 100644
--- a/asterix-app/src/test/resources/optimizerts/results/noncollocated.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/collocated.plan
@@ -5,14 +5,14 @@
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- HYBRID_HASH_JOIN [$$10][$$11] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$10] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$11] |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inlined_q18_large_volume_customer.plan b/asterix-app/src/test/resources/optimizerts/results/inlined_q18_large_volume_customer.plan
index 0c7b95d..8ae274a 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inlined_q18_large_volume_customer.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inlined_q18_large_volume_customer.plan
@@ -73,8 +73,8 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- EMPTY_TUPLE_SOURCE |PARTITIONED|
-- HASH_PARTITION_EXCHANGE [$$60] |PARTITIONED|
- -- STREAM_PROJECT |UNPARTITIONED|
- -- ASSIGN |UNPARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- SPLIT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
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/optimizerts/results/scan-delete-all.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan
index cba987c..c241913 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan
@@ -3,7 +3,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- INSERT_DELETE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$20(ASC)] |PARTITIONED|
+ -- MATERIALIZE |PARTITIONED|
-- HASH_PARTITION_EXCHANGE [$$20] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -11,4 +11,4 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
index e1852af..51800a2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
@@ -10,7 +10,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- INSERT_DELETE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$15(ASC)] |PARTITIONED|
+ -- MATERIALIZE |PARTITIONED|
-- HASH_PARTITION_EXCHANGE [$$15] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -20,4 +20,4 @@
-- BTREE_SEARCH |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan
index a5265c8..3a4bc84 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan
@@ -3,7 +3,7 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- INSERT_DELETE |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$22(ASC)] |PARTITIONED|
+ -- MATERIALIZE |PARTITIONED|
-- HASH_PARTITION_EXCHANGE [$$22] |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
@@ -13,4 +13,4 @@
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
index 8057666..cc9a865 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
@@ -12,15 +12,13 @@
-- ASSIGN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- INSERT_DELETE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$7(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$7] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$7] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan b/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan
index 51d3060..b34944d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan
@@ -2,15 +2,13 @@
-- STREAM_PROJECT |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
-- INSERT_DELETE |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- STABLE_SORT [$$7(ASC)] |PARTITIONED|
- -- HASH_PARTITION_EXCHANGE [$$7] |PARTITIONED|
+ -- HASH_PARTITION_EXCHANGE [$$7] |PARTITIONED|
+ -- ASSIGN |PARTITIONED|
+ -- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
-- STREAM_PROJECT |PARTITIONED|
-- ASSIGN |PARTITIONED|
- -- STREAM_PROJECT |PARTITIONED|
- -- ASSIGN |PARTITIONED|
+ -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
+ -- DATASOURCE_SCAN |PARTITIONED|
-- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- DATASOURCE_SCAN |PARTITIONED|
- -- ONE_TO_ONE_EXCHANGE |PARTITIONED|
- -- EMPTY_TUPLE_SOURCE |PARTITIONED|
+ -- EMPTY_TUPLE_SOURCE |PARTITIONED|
\ 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.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/queries/types/record01/record01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/types/record01/record01.1.ddl.aql
new file mode 100644
index 0000000..faae040
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/types/record01/record01.1.ddl.aql
@@ -0,0 +1,5 @@
+drop dataverse local if exists;
+create dataverse local;
+use dataverse local;
+create type ttype as { "id" : int32 } ;
+create dataset dset (ttype) primary key id;
\ No newline at end of file
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/results/types/record01/record01.1.adm b/asterix-app/src/test/resources/runtimets/results/types/record01/record01.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/types/record01/record01.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index eadf7fb..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>
@@ -4442,4 +4457,11 @@
</compilation-unit>
</test-case>
</test-group>
+ <test-group name="types">
+ <test-case FilePath="types">
+ <compilation-unit name="record01">
+ <output-dir compare="Text">record01</output-dir>
+ </compilation-unit>
+ </test-case>
+ </test-group>
</test-suite>
diff --git a/asterix-aql/src/main/javacc/AQL.jj b/asterix-aql/src/main/javacc/AQL.jj
index 84d46f5..fdcd86e 100644
--- a/asterix-aql/src/main/javacc/AQL.jj
+++ b/asterix-aql/src/main/javacc/AQL.jj
@@ -96,6 +96,49 @@
}
return s.substring(1).trim();
}
+
+ private static IRecordFieldDataGen parseFieldDataGen(String hint) throws ParseException {
+ IRecordFieldDataGen rfdg = null;
+ String splits[] = hint.split(" +");
+ if (splits[0].equals(VAL_FILE_HINT)) {
+ File[] valFiles = new File[splits.length - 1];
+ for (int k=1; k<splits.length; k++) {
+ valFiles[k-1] = new File(splits[k]);
+ }
+ rfdg = new FieldValFileDataGen(valFiles);
+ } else if (splits[0].equals(VAL_FILE_SAME_INDEX_HINT)) {
+ rfdg = new FieldValFileSameIndexDataGen(new File(splits[1]), splits[2]);
+ } else if (splits[0].equals(LIST_VAL_FILE_HINT)) {
+ rfdg = new ListValFileDataGen(new File(splits[1]), Integer.parseInt(splits[2]), Integer.parseInt(splits[3]));
+ } else if (splits[0].equals(LIST_HINT)) {
+ rfdg = new ListDataGen(Integer.parseInt(splits[1]), Integer.parseInt(splits[2]));
+ } else if (splits[0].equals(INTERVAL_HINT)) {
+ FieldIntervalDataGen.ValueType vt;
+ if (splits[1].equals("int")) {
+ vt = FieldIntervalDataGen.ValueType.INT;
+ } else if (splits[1].equals("long")) {
+ vt = FieldIntervalDataGen.ValueType.LONG;
+ } else if (splits[1].equals("float")) {
+ vt = FieldIntervalDataGen.ValueType.FLOAT;
+ } else if (splits[1].equals("double")) {
+ vt = FieldIntervalDataGen.ValueType.DOUBLE;
+ } else {
+ throw new ParseException("Unknown type for interval data gen: " + splits[1]);
+ }
+ rfdg = new FieldIntervalDataGen(vt, splits[2], splits[3]);
+ } else if (splits[0].equals(INSERT_RAND_INT_HINT)) {
+ rfdg = new InsertRandIntDataGen(splits[1], splits[2]);
+ } else if (splits[0].equals(DATE_BETWEEN_YEARS_HINT)) {
+ rfdg = new DateBetweenYearsDataGen(Integer.parseInt(splits[1]), Integer.parseInt(splits[2]));
+ } else if (splits[0].equals(DATETIME_BETWEEN_YEARS_HINT)) {
+ rfdg = new DatetimeBetweenYearsDataGen(Integer.parseInt(splits[1]), Integer.parseInt(splits[2]));
+ } else if (splits[0].equals(DATETIME_ADD_RAND_HOURS_HINT)) {
+ rfdg = new DatetimeAddRandHoursDataGen(Integer.parseInt(splits[1]), Integer.parseInt(splits[2]), splits[3]);
+ } else if (splits[0].equals(AUTO_HINT)) {
+ rfdg = new AutoDataGen(splits[1]);
+ }
+ return rfdg;
+ }
public AQLParser(String s){
this(new StringReader(s));
@@ -893,63 +936,20 @@
void RecordField(RecordTypeDefinition recType) throws ParseException:
{
- String fieldName;
- TypeExpression type = null;
- boolean nullable = false;
+ String fieldName;
+ TypeExpression type = null;
+ boolean nullable = false;
}
{
- fieldName = Identifier()
- {
- fieldName = token.image;
- String hint = getHint(token);
- IRecordFieldDataGen rfdg = null;
- if (hint != null) {
- String splits[] = hint.split(" +");
- if (splits[0].equals(VAL_FILE_HINT)) {
- File[] valFiles = new File[splits.length - 1];
- for (int k=1; k<splits.length; k++) {
- valFiles[k-1] = new File(splits[k]);
- }
- rfdg = new FieldValFileDataGen(valFiles);
- } else if (splits[0].equals(VAL_FILE_SAME_INDEX_HINT)) {
- rfdg = new FieldValFileSameIndexDataGen(new File(splits[1]), splits[2]);
- } else if (splits[0].equals(LIST_VAL_FILE_HINT)) {
- rfdg = new ListValFileDataGen(new File(splits[1]), Integer.parseInt(splits[2]), Integer.parseInt(splits[3]));
- } else if (splits[0].equals(LIST_HINT)) {
- rfdg = new ListDataGen(Integer.parseInt(splits[1]), Integer.parseInt(splits[2]));
- } else if (splits[0].equals(INTERVAL_HINT)) {
- FieldIntervalDataGen.ValueType vt;
- if (splits[1].equals("int")) {
- vt = FieldIntervalDataGen.ValueType.INT;
- } else if (splits[1].equals("long")) {
- vt = FieldIntervalDataGen.ValueType.LONG;
- } else if (splits[1].equals("float")) {
- vt = FieldIntervalDataGen.ValueType.FLOAT;
- } else if (splits[1].equals("double")) {
- vt = FieldIntervalDataGen.ValueType.DOUBLE;
- } else {
- throw new ParseException("Unknown type for interval data gen: " + splits[1]);
- }
- rfdg = new FieldIntervalDataGen(vt, splits[2], splits[3]);
- } else if (splits[0].equals(INSERT_RAND_INT_HINT)) {
- rfdg = new InsertRandIntDataGen(splits[1], splits[2]);
- } else if (splits[0].equals(DATE_BETWEEN_YEARS_HINT)) {
- rfdg = new DateBetweenYearsDataGen(Integer.parseInt(splits[1]), Integer.parseInt(splits[2]));
- } else if (splits[0].equals(DATETIME_BETWEEN_YEARS_HINT)) {
- rfdg = new DatetimeBetweenYearsDataGen(Integer.parseInt(splits[1]), Integer.parseInt(splits[2]));
- } else if (splits[0].equals(DATETIME_ADD_RAND_HOURS_HINT)) {
- rfdg = new DatetimeAddRandHoursDataGen(Integer.parseInt(splits[1]), Integer.parseInt(splits[2]), splits[3]);
- } else if (splits[0].equals(AUTO_HINT)) {
- rfdg = new AutoDataGen(splits[1]);
- }
- }
- }
- ":"
- ( type = TypeExpr() )
- ("?" { nullable = true; } )?
- {
- recType.addField(fieldName, type, nullable, rfdg);
- }
+ fieldName = Identifier()
+ {
+ String hint = getHint(token);
+ IRecordFieldDataGen rfdg = hint != null ? parseFieldDataGen(hint) : null;
+ }
+ ":" type = TypeExpr() ("?" { nullable = true; } )?
+ {
+ recType.addField(fieldName, type, nullable, rfdg);
+ }
}
TypeReferenceExpression TypeReference() throws ParseException:
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
index 6fa5beb..5a40ece 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
@@ -23,7 +23,7 @@
private static final int TXN_LOG_BUFFER_PAGESIZE_DEFAULT = (128 << 10); // 128KB
private static final String TXN_LOG_PARTITIONSIZE_KEY = "txn.log.partitionsize";
- private static final long TXN_LOG_PARTITIONSIZE_DEFAULT = (2 << 30); // 2GB
+ private static final long TXN_LOG_PARTITIONSIZE_DEFAULT = ((long)2 << 30); // 2GB
private static final String TXN_LOG_DISKSECTORSIZE_KEY = "txn.log.disksectorsize";
private static final int TXN_LOG_DISKSECTORSIZE_DEFAULT = 4096;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java
index 7ef378e..fbe9b4c 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java
@@ -83,7 +83,7 @@
buffer.position(pos);
buffer.limit(limit);
fileChannel.write(buffer);
- fileChannel.force(true);
+ fileChannel.force(false);
//update variables
bufferLastFlushOffset = limit;
diff --git a/asterix-installer/ittest/asterix-lifecycle_backupRestore.adm b/asterix-installer/ittest/asterix-lifecycle_backupRestore.adm
deleted file mode 100644
index 4c4f073..0000000
--- a/asterix-installer/ittest/asterix-lifecycle_backupRestore.adm
+++ /dev/null
@@ -1 +0,0 @@
-{ "DataverseName": "backupDataverse", "DataFormat": "edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat", "Timestamp": "Mon Jul 08 17:23:56 PDT 2013", "PendingOp": 0 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
index e62cc59..2e04bbf 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
@@ -247,7 +247,8 @@
String txnLogDir = null;
for (Node node : cluster.getNode()) {
coredumpDir = node.getLogDir() == null ? cluster.getLogDir() : node.getLogDir();
- coredump.add(new Coredump(asterixInstanceName + "_" + node.getId(), coredumpDir + File.separator + asterixInstanceName + "_" + node.getId()));
+ coredump.add(new Coredump(asterixInstanceName + "_" + node.getId(), coredumpDir + File.separator
+ + asterixInstanceName + "_" + node.getId()));
txnLogDir = node.getTxnLogDir() == null ? cluster.getTxnLogDir() : node.getTxnLogDir();
txnLogDirs.add(new TransactionLogDir(asterixInstanceName + "_" + node.getId(), txnLogDir));
@@ -455,17 +456,19 @@
boolean conflictFound = false;
AsterixInstance conflictingInstance = null;
for (AsterixInstance existing : existingInstances) {
- conflictFound = existing.getCluster().getMasterNode().getClusterIp().equals(masterIp);
- if (conflictFound) {
- conflictingInstance = existing;
- break;
- }
- for (Node n : existing.getCluster().getNode()) {
- if (usedIps.contains(n.getClusterIp())) {
- conflictFound = true;
+ if (!existing.getState().equals(State.INACTIVE)) {
+ conflictFound = existing.getCluster().getMasterNode().getClusterIp().equals(masterIp);
+ if (conflictFound) {
conflictingInstance = existing;
break;
}
+ for (Node n : existing.getCluster().getNode()) {
+ if (usedIps.contains(n.getClusterIp())) {
+ conflictFound = true;
+ conflictingInstance = existing;
+ break;
+ }
+ }
}
}
if (conflictFound) {
diff --git a/asterix-installer/src/main/resources/conf/asterix-configuration.xml b/asterix-installer/src/main/resources/conf/asterix-configuration.xml
index e542efd..6b3133c 100644
--- a/asterix-installer/src/main/resources/conf/asterix-configuration.xml
+++ b/asterix-installer/src/main/resources/conf/asterix-configuration.xml
@@ -126,9 +126,9 @@
<property>
<name>txn.log.groupcommitinterval</name>
- <value>1</value>
+ <value>40</value>
<description>The group commit wait time in milliseconds. (Default =
- "10" // 0.1ms)
+ "40" // 40ms)
</description>
</property>
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
index 0ed3c78..166de00 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
@@ -32,7 +32,7 @@
import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
-import edu.uci.ics.hyracks.algebricks.core.algebra.properties.AsterixNodeGroupDomain;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.DefaultNodeGroupDomain;
import edu.uci.ics.hyracks.algebricks.core.algebra.properties.FunctionalDependency;
import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
import edu.uci.ics.hyracks.algebricks.core.algebra.properties.INodeDomain;
@@ -120,7 +120,7 @@
schemaTypes[i] = recordType.getFieldType(partitioningKeys.get(i));
}
schemaTypes[n] = itemType;
- domain = new AsterixNodeGroupDomain(DatasetUtils.getNodegroupName(dataset));
+ domain = new DefaultNodeGroupDomain(DatasetUtils.getNodegroupName(dataset));
}
private void initFeedDataset(IAType itemType, Dataset dataset) throws IOException {
@@ -135,7 +135,7 @@
schemaTypes[i] = recordType.getFieldType(partitioningKeys.get(i));
}
schemaTypes[n] = itemType;
- domain = new AsterixNodeGroupDomain(DatasetUtils.getNodegroupName(dataset));
+ domain = new DefaultNodeGroupDomain(DatasetUtils.getNodegroupName(dataset));
}
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
index a18b3d3..b09cd52 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
@@ -32,9 +32,9 @@
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
import edu.uci.ics.asterix.common.transactions.JobId;
+import edu.uci.ics.asterix.common.transactions.LogUtil;
import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogUtil;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java
index 9c9439c..755e85b 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java
@@ -24,6 +24,7 @@
import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
+import edu.uci.ics.asterix.common.transactions.LogUtil;
import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
index 645b613..d3e1ad1 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
@@ -24,6 +24,7 @@
import edu.uci.ics.asterix.common.transactions.ILogCursor;
import edu.uci.ics.asterix.common.transactions.ILogFilter;
import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
+import edu.uci.ics.asterix.common.transactions.LogUtil;
import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
@@ -120,7 +121,7 @@
// indicates an absence of logs any further.
}
- if (logicalLogLocator.getLsn() > logManager.getLastFlushedLsn().get()) {
+ if (logManager.isMemoryRead(logicalLogLocator.getLsn())) {
return next(currentLogLocator); //should read from memory if there is any further log
}
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
index d4eb8a4..5b10144 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
@@ -16,7 +16,6 @@
import java.io.File;
import java.io.IOException;
-import java.io.InputStream;
import java.io.OutputStream;
import java.io.RandomAccessFile;
import java.nio.ByteBuffer;
@@ -25,7 +24,6 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import java.util.Properties;
import java.util.Set;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicInteger;
@@ -45,10 +43,10 @@
import edu.uci.ics.asterix.common.transactions.ILogger;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
+import edu.uci.ics.asterix.common.transactions.LogUtil;
import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
@@ -591,14 +589,14 @@
readDiskLog(lsnValue, logicalLogLocator);
}
- public boolean isMemoryRead(long currentLSN) {
+ public boolean isMemoryRead(long readLSN) {
long flushLSN = lastFlushedLSN.get();
- if ((flushLSN + 1) % logPageSize == 0) {
+ if ((flushLSN + 1) == readLSN) {
return false;
}
long logPageBeginOffset = flushLSN - (flushLSN % logPageSize);
long logPageEndOffset = logPageBeginOffset + logPageSize;
- if (currentLSN > flushLSN || (currentLSN >= logPageBeginOffset && currentLSN < logPageEndOffset)) {
+ if (readLSN > flushLSN || (readLSN >= logPageBeginOffset && readLSN < logPageEndOffset)) {
return true;
} else {
return false;
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogUtil.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogUtil.java
deleted file mode 100644
index 035a319..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogUtil.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * 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.transaction.management.service.logging;
-
-import java.io.File;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.FileUtil;
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-
-
-/**
- * A utility class providing helper methods for the {@link ILogManager}
- */
-public class LogUtil {
-
- private static final Logger LOGGER = Logger.getLogger(LogUtil.class.getName());
-
- // read the log directory and initialize log anchor to point to the
- // current log partition file and the offset where the log manager shall
- // continue to insert log records.
-
- public static PhysicalLogLocator initializeLogAnchor(ILogManager logManager) throws ACIDException {
- int fileId = 0;
- long offset = 0;
- LogManagerProperties logManagerProperties = logManager.getLogManagerProperties();
- File logDir = new File(logManagerProperties.getLogDir());
- try {
- if (logDir.exists()) {
- List<String> logFiles = getLogFiles(logManagerProperties);
- if (logFiles == null || logFiles.size() == 0) {
- FileUtil.createFileIfNotExists(getLogFilePath(logManagerProperties, 0));
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("created a log file: " + getLogFilePath(logManagerProperties, 0));
- }
- } else {
- File logFile = new File(LogUtil.getLogFilePath(logManagerProperties,
- Long.parseLong(logFiles.get(logFiles.size() - 1))));
- fileId = logFiles.size() - 1;
- offset = logFile.length();
- }
- } else {
- FileUtil.createNewDirectory(logManagerProperties.getLogDir());
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("created the log directory: " + logManagerProperties.getLogDir());
- }
- FileUtil.createFileIfNotExists(getLogFilePath(logManagerProperties, 0));
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info("created a log file: " + getLogFilePath(logManagerProperties, 0));
- }
- }
- } catch (IOException ioe) {
- throw new ACIDException("Unable to initialize log anchor", ioe);
- }
- if (LOGGER.isLoggable(Level.INFO)) {
- LOGGER.info(" file id :" + fileId + " offset " + offset);
- }
- return new PhysicalLogLocator(fileId, offset, logManager);
- }
-
- public static List<String> getLogFiles(final LogManagerProperties logManagerProperties) {
- File logDir = new File(logManagerProperties.getLogDir());
- String[] logFiles = new String[0];
- List<String> logFileCollection = new ArrayList<String>();
- if (logDir.exists()) {
- logFiles = logDir.list(new FilenameFilter() {
- public boolean accept(File dir, String name) {
- if (name.startsWith(logManagerProperties.getLogFilePrefix())) {
- return true;
- }
- return false;
- }
- });
- }
- for (String logFile : logFiles) {
- logFileCollection.add(logFile.substring(logManagerProperties.getLogFilePrefix().length() + 1));
- }
- Collections.sort(logFileCollection, new Comparator<String>() {
- @Override
- public int compare(String arg0, String arg1) {
- return Integer.parseInt(arg0) - Integer.parseInt(arg1);
- }
- });
- return logFileCollection;
- }
-
- public static long getFileId(String logFilePath, LogManagerProperties logManagerProperties) {
- String logFileName = logFilePath;
- if (logFilePath.contains(File.separator)) {
- logFileName = logFilePath.substring(logFilePath.lastIndexOf(File.separator));
- }
- return Long.parseLong(logFileName.substring(logFileName.indexOf(logManagerProperties.getLogFilePrefix())));
- }
-
- public static String getLogFilePath(LogManagerProperties logManagerProperties, long fileId) {
- return logManagerProperties.getLogDir() + File.separator + logManagerProperties.getLogFilePrefix() + "_"
- + fileId;
- }
-
- public static LogicalLogLocator getDummyLogicalLogLocator(ILogManager logManager) {
- LogicalLogLocator logicalLogLocator = new LogicalLogLocator(-1, null, -1, logManager);
- return logicalLogLocator;
- }
-
- /*
- * given a lsn, get the offset within the log file where the corresponding
- * log record is (to be) placed.
- */
- public static long getFileOffset(ILogManager logManager, long lsn) {
- return lsn % logManager.getLogManagerProperties().getLogPartitionSize();
- }
-
- /*
- * given a lsn, get the file id that contains the log record.
- */
- public static long getFileId(ILogManager logManager, long lsn) {
- return lsn / logManager.getLogManagerProperties().getLogPartitionSize();
- }
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
index 5b946bc..ad1db1f 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
@@ -49,12 +49,12 @@
import edu.uci.ics.asterix.common.transactions.IResourceManager;
import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.LogUtil;
import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
import edu.uci.ics.asterix.transaction.management.service.logging.IndexResourceManager;
import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogUtil;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManager;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
index 06a5fbf..33522e3 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
@@ -26,8 +26,8 @@
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
import edu.uci.ics.asterix.common.transactions.JobId;
+import edu.uci.ics.asterix.common.transactions.LogUtil;
import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogUtil;
import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
index 1298791..a74b079 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
@@ -28,11 +28,11 @@
import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
import edu.uci.ics.asterix.common.transactions.IResourceManager;
import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.LogUtil;
import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
import edu.uci.ics.asterix.transaction.management.logging.BasicLogger;
import edu.uci.ics.asterix.transaction.management.service.logging.LogActionType;
import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogUtil;
import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java
index 9aed571..8e13356 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java
@@ -22,10 +22,10 @@
import edu.uci.ics.asterix.common.transactions.ILogFilter;
import edu.uci.ics.asterix.common.transactions.ILogManager;
import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
+import edu.uci.ics.asterix.common.transactions.LogUtil;
import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogUtil;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
public class LogRecordReader {
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
index b683a93..4bebbc4 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
@@ -31,10 +31,10 @@
import edu.uci.ics.asterix.common.transactions.ITransactionContext;
import edu.uci.ics.asterix.common.transactions.ITransactionManager;
import edu.uci.ics.asterix.common.transactions.JobId;
+import edu.uci.ics.asterix.common.transactions.LogUtil;
import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
import edu.uci.ics.asterix.transaction.management.logging.IResource;
import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogUtil;
import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
diff --git a/pom.xml b/pom.xml
index 808f075..308900a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -21,8 +21,8 @@
<packaging>pom</packaging>
<properties>
- <algebricks.version>0.2.7-SNAPSHOT</algebricks.version>
- <hyracks.version>0.2.7-SNAPSHOT</hyracks.version>
+ <algebricks.version>0.2.10-SNAPSHOT</algebricks.version>
+ <hyracks.version>0.2.10-SNAPSHOT</hyracks.version>
</properties>
<build>