[ASTERIXDB-2626][*DB] Ignore dup fields in record construction, issue warning
- user model changes: yes
- storage format changes: no
- interface changes: no
Details:
Change the current behavior of throwing exceptions on duplicates fields
on a record to just picking one and return a warning when possible.
- added source location for auto generated name expressions.
- removed duplicate test cases from AQL test suite since SQL++ alraedy has it.
- updated test cases that used to throw exception on duplicate fields.
- updated an AST test case to reflect the new behaviour of picking one field.
- clean-ups in testsuite_sqlpp:
moved flwor let33 test case to its proper group.
moved ObjectsQueries, GeoQueries xml references up to where other xml refs.
Change-Id: I2d773b194f0e0f35720bb3fd63f1997239800762
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3519
Contrib: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Ali Alsuliman <ali.al.solaiman@gmail.com>
Reviewed-by: Dmitry Lychagin <dmitry.lychagin@couchbase.com>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index d5f2e00..318b18c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -74,6 +74,7 @@
import org.apache.asterix.optimizer.rules.PushLimitIntoPrimarySearchRule;
import org.apache.asterix.optimizer.rules.PushProperJoinThroughProduct;
import org.apache.asterix.optimizer.rules.PushSimilarityFunctionsBelowJoin;
+import org.apache.asterix.optimizer.rules.RemoveDuplicateFieldsRule;
import org.apache.asterix.optimizer.rules.RemoveLeftOuterUnnestForLeftOuterJoinRule;
import org.apache.asterix.optimizer.rules.RemoveRedundantListifyRule;
import org.apache.asterix.optimizer.rules.RemoveRedundantSelectRule;
@@ -151,10 +152,11 @@
private RuleCollections() {
}
- public static final List<IAlgebraicRewriteRule> buildInitialTranslationRuleCollection() {
+ public static List<IAlgebraicRewriteRule> buildInitialTranslationRuleCollection() {
List<IAlgebraicRewriteRule> translationRules = new LinkedList<>();
translationRules.add(new TranslateIntervalExpressionRule());
translationRules.add(new ExtractGroupByDecorVariablesRule());
+ translationRules.add(new RemoveDuplicateFieldsRule());
return translationRules;
}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index 7a63ea3..c8751f1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -21,12 +21,16 @@
import java.io.DataInputStream;
import java.nio.ByteBuffer;
+import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.asterix.common.config.GlobalConfig;
import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.common.exceptions.WarningCollector;
+import org.apache.asterix.common.exceptions.WarningUtil;
import org.apache.asterix.dataflow.data.common.ExpressionTypeComputer;
import org.apache.asterix.dataflow.data.nontagged.MissingWriterFactory;
import org.apache.asterix.formats.nontagged.ADMPrinterFactoryProvider;
@@ -213,7 +217,9 @@
}
IAObject c = FUNC_ID_TO_CONSTANT.get(expr.getFunctionIdentifier());
if (c != null) {
- return new Pair<>(true, new ConstantExpression(new AsterixConstantValue(c)));
+ ConstantExpression constantExpression = new ConstantExpression(new AsterixConstantValue(c));
+ constantExpression.setSourceLocation(expr.getSourceLocation());
+ return new Pair<>(true, constantExpression);
}
IScalarEvaluatorFactory fact = jobGenCtx.getExpressionRuntimeProvider().createEvaluatorFactory(expr,
@@ -235,7 +241,9 @@
bbis.setByteBuffer(ByteBuffer.wrap(p.getByteArray(), p.getStartOffset(), p.getLength()), 0);
IAObject o = (IAObject) serde.deserialize(dis);
warningCollector.getWarnings(optContext.getWarningCollector());
- return new Pair<>(true, new ConstantExpression(new AsterixConstantValue(o)));
+ ConstantExpression constantExpression = new ConstantExpression(new AsterixConstantValue(o));
+ constantExpression.setSourceLocation(expr.getSourceLocation());
+ return new Pair<>(true, constantExpression);
} catch (HyracksDataException | AlgebricksException e) {
if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Exception caught at constant folding: " + e, e);
@@ -266,17 +274,75 @@
}
private boolean constantFoldArgs(AbstractFunctionCallExpression expr, Void arg) throws AlgebricksException {
+ return expr.getFunctionIdentifier().equals(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR)
+ ? foldRecordArgs(expr, arg) : foldFunctionArgs(expr, arg);
+ }
+
+ private boolean foldFunctionArgs(AbstractFunctionCallExpression expr, Void arg) throws AlgebricksException {
boolean changed = false;
- for (Mutable<ILogicalExpression> r : expr.getArguments()) {
- Pair<Boolean, ILogicalExpression> p2 = r.getValue().accept(this, arg);
- if (p2.first) {
- r.setValue(p2.second);
+ for (Mutable<ILogicalExpression> exprArgRef : expr.getArguments()) {
+ changed |= foldArg(exprArgRef, arg);
+ }
+ return changed;
+ }
+
+ private boolean foldRecordArgs(AbstractFunctionCallExpression expr, Void arg) throws AlgebricksException {
+ if (expr.getArguments().size() % 2 != 0) {
+ String functionName = expr.getFunctionIdentifier().getName();
+ throw CompilationException.create(ErrorCode.COMPILATION_INVALID_NUM_OF_ARGS, expr.getSourceLocation(),
+ functionName);
+ }
+ boolean changed = false;
+ Iterator<Mutable<ILogicalExpression>> iterator = expr.getArguments().iterator();
+ int fieldNameIdx = 0;
+ while (iterator.hasNext()) {
+ Mutable<ILogicalExpression> fieldNameExprRef = iterator.next();
+ Pair<Boolean, ILogicalExpression> fieldNameExpr = fieldNameExprRef.getValue().accept(this, arg);
+ boolean isDuplicate = false;
+ if (fieldNameExpr.first) {
+ String fieldName = ConstantExpressionUtil.getStringConstant(fieldNameExpr.second);
+ if (fieldName != null) {
+ isDuplicate = isDuplicateField(fieldName, fieldNameIdx, expr.getArguments());
+ }
+ if (isDuplicate) {
+ optContext.getWarningCollector()
+ .warn(WarningUtil.forAsterix(fieldNameExpr.second.getSourceLocation(),
+ ErrorCode.COMPILATION_DUPLICATE_FIELD_NAME, fieldName));
+ iterator.remove();
+ iterator.next();
+ iterator.remove();
+ } else {
+ fieldNameExprRef.setValue(fieldNameExpr.second);
+ }
changed = true;
}
+ if (!isDuplicate) {
+ Mutable<ILogicalExpression> fieldValue = iterator.next();
+ changed |= foldArg(fieldValue, arg);
+ fieldNameIdx += 2;
+ }
}
return changed;
}
+ private boolean isDuplicateField(String fName, int fIdx, List<Mutable<ILogicalExpression>> args) {
+ for (int i = 0, size = args.size(); i < size; i += 2) {
+ if (i != fIdx && fName.equals(ConstantExpressionUtil.getStringConstant(args.get(i).getValue()))) {
+ return true;
+ }
+ }
+ return false;
+ }
+
+ private boolean foldArg(Mutable<ILogicalExpression> exprArgRef, Void arg) throws AlgebricksException {
+ Pair<Boolean, ILogicalExpression> newExpr = exprArgRef.getValue().accept(this, arg);
+ if (newExpr.first) {
+ exprArgRef.setValue(newExpr.second);
+ return true;
+ }
+ return false;
+ }
+
private boolean allArgsConstant(AbstractFunctionCallExpression expr) {
for (Mutable<ILogicalExpression> r : expr.getArguments()) {
if (r.getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveDuplicateFieldsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveDuplicateFieldsRule.java
new file mode 100644
index 0000000..d457ce5
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveDuplicateFieldsRule.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.optimizer.rules;
+
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.WarningUtil;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.typecomputer.impl.ClosedRecordConstructorResultType;
+import org.apache.asterix.om.typecomputer.impl.OpenRecordConstructorResultType;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * <pre>
+ * This rule removes duplicate fields from record constructors. For example:
+ * {"f1": 87, "f2": "val2", "f1": "str"}.
+ * For records where the field name is not a constant string, {@link ConstantFoldingRule} will evaluate the
+ * expression and remove the field if it is a duplicate. If the field name expression cannot be constant folded, then
+ * {@link RecordBuilder} will take care of not adding the field at runtime if it is a duplicate. Such field names
+ * will be added in the open part of the record. Examples:
+ * <ol>
+ * <li>{"f1": 11, lowercase("F1"): 12}</li>
+ * <li>{"f1": 11, lowercase($$var): 12}</li>
+ * </ol>
+ *
+ * Note: {@link OpenRecordConstructorResultType} and {@link ClosedRecordConstructorResultType} still have a sanity
+ * check that tests that there are no duplicate fields during compilation.
+ * </pre>
+ */
+public class RemoveDuplicateFieldsRule implements IAlgebraicRewriteRule {
+
+ private final Set<String> fieldNames = new HashSet<>();
+ private IOptimizationContext context;
+
+ @Override
+ public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext ctx) throws AlgebricksException {
+ return false;
+ }
+
+ @Override
+ public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext ctx) throws AlgebricksException {
+ if (ctx.checkIfInDontApplySet(this, opRef.getValue())) {
+ // children should've already been visited and marked through the root operator
+ return false;
+ }
+ context = ctx;
+ return rewriteOpAndInputs(opRef, true);
+ }
+
+ private boolean rewriteOpAndInputs(Mutable<ILogicalOperator> opRef, boolean first) throws AlgebricksException {
+ AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+ if (!first) {
+ context.addToDontApplySet(this, op);
+ }
+ List<Mutable<ILogicalOperator>> inputs = op.getInputs();
+ boolean changed = false;
+ for (int i = 0, size = inputs.size(); i < size; i++) {
+ changed |= rewriteOpAndInputs(inputs.get(i), false);
+ }
+ changed |= op.acceptExpressionTransform(this::transform);
+ return changed;
+ }
+
+ private boolean transform(Mutable<ILogicalExpression> expressionRef) throws AlgebricksException {
+ ILogicalExpression expr = expressionRef.getValue();
+ if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+ return false;
+ }
+ AbstractFunctionCallExpression function = (AbstractFunctionCallExpression) expr;
+ boolean changed = false;
+ if (function.getFunctionIdentifier().equals(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR)
+ || function.getFunctionIdentifier().equals(BuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR)) {
+ if (function.getArguments().size() % 2 != 0) {
+ String functionName = function.getFunctionIdentifier().getName();
+ throw CompilationException.create(ErrorCode.COMPILATION_INVALID_NUM_OF_ARGS, expr.getSourceLocation(),
+ functionName);
+ }
+ fieldNames.clear();
+ Iterator<Mutable<ILogicalExpression>> iterator = function.getArguments().iterator();
+ while (iterator.hasNext()) {
+ ILogicalExpression fieldNameExpr = iterator.next().getValue();
+ String fieldName = ConstantExpressionUtil.getStringConstant(fieldNameExpr);
+ if (fieldName != null && !fieldNames.add(fieldName)) {
+ context.getWarningCollector().warn(WarningUtil.forAsterix(fieldNameExpr.getSourceLocation(),
+ ErrorCode.COMPILATION_DUPLICATE_FIELD_NAME, fieldName));
+ iterator.remove();
+ iterator.next();
+ iterator.remove();
+ changed = true;
+ } else {
+ iterator.next();
+ }
+ }
+ }
+ List<Mutable<ILogicalExpression>> arguments = function.getArguments();
+ for (int i = 0, size = arguments.size(); i < size; i++) {
+ changed |= transform(arguments.get(i));
+ }
+ return changed;
+ }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/ObjectsQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/ObjectsQueries.xml
index 4d9403d..87af8c4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/ObjectsQueries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/ObjectsQueries.xml
@@ -112,24 +112,6 @@
<output-dir compare="Text">open-object-constructor_02</output-dir>
</compilation-unit>
</test-case>
- <test-case FilePath="objects">
- <compilation-unit name="closed-closed-fieldname-conflict_issue173">
- <output-dir compare="Text">closed-closed-fieldname-conflict_issue173</output-dir>
- <expected-error>Duplicate field name "name"</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="objects">
- <compilation-unit name="open-closed-fieldname-conflict_issue173">
- <output-dir compare="Text">open-closed-fieldname-conflict_issue173</output-dir>
- <expected-error>Open field "name" has the same field name as closed field at index 0</expected-error>
- </compilation-unit>
- </test-case>
- <test-case FilePath="objects">
- <compilation-unit name="open-open-fieldname-conflict_issue173">
- <output-dir compare="Text">open-open-fieldname-conflict_issue173</output-dir>
- <expected-error>Open fields 0 and 1 have the same field name "name"</expected-error>
- </compilation-unit>
- </test-case>
<!-- RECORD MANIPULATION TESTS -->
<test-case FilePath="objects/object-merge">
<compilation-unit name="documentation-example">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.1.ddl.aql
deleted file mode 100644
index 540d649..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.1.ddl.aql
+++ /dev/null
@@ -1,25 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Tests whether a conflict between two closed field names are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
- * Author: zheilbron
- */
-drop dataverse test if exists;
-create dataverse test;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.3.query.aql
deleted file mode 100644
index bf5a986..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.3.query.aql
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Tests whether a conflict between two closed field names are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
- * Author: zheilbron
- */
-use dataverse test;
-
-let $x := {"name": "john", "name": "smith"}
-return $x
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-closed-fieldname-conflict_issue173/open-closed-fieldname-conflict_issue173.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-closed-fieldname-conflict_issue173/open-closed-fieldname-conflict_issue173.1.ddl.aql
deleted file mode 100644
index f5890e3..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-closed-fieldname-conflict_issue173/open-closed-fieldname-conflict_issue173.1.ddl.aql
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Tests whether a conflict between an open and closed field name are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
- * Author: zheilbron
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type opentype as open {
-id:int32,
-fname:string
-}
-
-create dataset testds(opentype) primary key id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-closed-fieldname-conflict_issue173/open-closed-fieldname-conflict_issue173.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-closed-fieldname-conflict_issue173/open-closed-fieldname-conflict_issue173.2.update.aql
deleted file mode 100644
index 6fcf620..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-closed-fieldname-conflict_issue173/open-closed-fieldname-conflict_issue173.2.update.aql
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Tests whether a conflict between an open and closed field name are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
- * Author: zheilbron
- */
-
-use dataverse test;
-
-insert into dataset testds({'id': 1, 'fname': "name"});
-
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-closed-fieldname-conflict_issue173/open-closed-fieldname-conflict_issue173.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-closed-fieldname-conflict_issue173/open-closed-fieldname-conflict_issue173.3.query.aql
deleted file mode 100644
index 074cd9b..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-closed-fieldname-conflict_issue173/open-closed-fieldname-conflict_issue173.3.query.aql
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Tests whether a conflict between an open and closed field name are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
- * Author: zheilbron
- */
-
-use dataverse test;
-
-for $x in dataset('testds')
-return {$x.fname: "smith", lowercase("NAME"): "john"}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-open-fieldname-conflict_issue173/open-open-fieldname-conflict_issue173.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-open-fieldname-conflict_issue173/open-open-fieldname-conflict_issue173.1.ddl.aql
deleted file mode 100644
index 6a4e4c9..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-open-fieldname-conflict_issue173/open-open-fieldname-conflict_issue173.1.ddl.aql
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Tests whether a conflict between two open field names are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
- * Author: zheilbron
- */
-
-drop dataverse test if exists;
-create dataverse test;
-use dataverse test;
-
-create type opentype as open {
-fname1: string,
-fname2: string
-}
-
-create dataset testds(opentype) primary key fname1;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-open-fieldname-conflict_issue173/open-open-fieldname-conflict_issue173.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-open-fieldname-conflict_issue173/open-open-fieldname-conflict_issue173.2.update.aql
deleted file mode 100644
index 9eba6d9..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-open-fieldname-conflict_issue173/open-open-fieldname-conflict_issue173.2.update.aql
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Tests whether a conflict between two open field names are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
- * Author: zheilbron
- */
-
-use dataverse test;
-
-insert into dataset testds({'fname1': "name", 'fname2': "name"});
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-open-fieldname-conflict_issue173/open-open-fieldname-conflict_issue173.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-open-fieldname-conflict_issue173/open-open-fieldname-conflict_issue173.3.query.aql
deleted file mode 100644
index 376a7fa..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/open-open-fieldname-conflict_issue173/open-open-fieldname-conflict_issue173.3.query.aql
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-/*
- * Description : Tests whether a conflict between two open field names are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
- * Author: zheilbron
- */
-
-use dataverse test;
-
-for $x in dataset('testds')
-return {$x.fname1: "john", $x.fname2: "smith"}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.2.query.sqlpp
index b548b5d..572acff 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.2.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.2.query.sqlpp
@@ -18,9 +18,9 @@
*/
/*
- * Description : Ensure error if there's a duplicate field name in the closed-object-constructor() function
+ * Description : Warn and ignore duplicates if there's a duplicate field name in the closed-object-constructor() function
*/
set `import-private-functions` `true`;
-`closed-object-constructor`('b',get_year(current_date()),'c',[20],'c',[30]);
\ No newline at end of file
+`closed-object-constructor`('b',tobigint(get_year(current_date()) / 1000),'c',[20],'c',[30]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.3.query.sqlpp
index 73810d8..23f718f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.3.query.sqlpp
@@ -18,9 +18,9 @@
*/
/*
- * Description : Ensure error if there's a duplicate field name in the open-object-constructor() function
+ * Description : Warn and ignore duplicates if there's a duplicate field name in the open-object-constructor() function
*/
set `import-private-functions` `true`;
-`open-object-constructor`('d' || to_string(get_year(current_date())),10,'e',[20],'e',[30]);
+`open-object-constructor`('d' || to_string((get_year(current_date()) + 1) - get_year(current_date())), 10,'e',[20],'e',[30]);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
index e37dee9c..3f8860f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/ObjectsQueries.xml
@@ -102,16 +102,17 @@
<output-dir compare="Text">tiny-social-example</output-dir>
</compilation-unit>
</test-case>
- <test-case FilePath="objects">
+ <test-case FilePath="objects" check-warnings="true">
<compilation-unit name="no_fieldname_constr">
<output-dir compare="Text">no_fieldname_constr</output-dir>
+ <expected-warn>Encountered a cross product join (in line 25, at column 22)</expected-warn>
+ <expected-warn>Duplicate field name "a" (in line 29, at column 24)</expected-warn>
</compilation-unit>
</test-case>
<test-case FilePath="objects">
<compilation-unit name="no_fieldname_constr_negative">
<output-dir compare="Text">no_fieldname_constr</output-dir>
<expected-error>ASX1001: Syntax error: Cannot infer field name</expected-error>
- <expected-error>ASX0013: Duplicate field name "a"</expected-error>
</compilation-unit>
</test-case>
<test-case FilePath="objects">
@@ -150,7 +151,7 @@
<source-location>false</source-location>
</compilation-unit>
</test-case>
- <test-case FilePath="objects">>
+ <test-case FilePath="objects">
<compilation-unit name="object_add">
<output-dir compare="Text">object_add</output-dir>
</compilation-unit>
@@ -185,24 +186,21 @@
<output-dir compare="Text">open-object-constructor_02</output-dir>
</compilation-unit>
</test-case>
- <test-case FilePath="objects">
+ <test-case FilePath="objects" check-warnings="true">
<compilation-unit name="closed-closed-fieldname-conflict_issue173">
<output-dir compare="Text">closed-closed-fieldname-conflict_issue173</output-dir>
- <expected-error>Duplicate field name "name"</expected-error>
- <source-location>false</source-location>
+ <expected-warn>Duplicate field name "name" (in line 28, at column 16)</expected-warn>
</compilation-unit>
</test-case>
- <test-case FilePath="objects">
+ <test-case FilePath="objects" check-warnings="true">
<compilation-unit name="open-closed-fieldname-conflict_issue173">
<output-dir compare="Text">open-closed-fieldname-conflict_issue173</output-dir>
- <expected-error>Open field "name" has the same field name as closed field at index 0</expected-error>
<source-location>false</source-location>
</compilation-unit>
</test-case>
- <test-case FilePath="objects">
+ <test-case FilePath="objects" check-warnings="true">
<compilation-unit name="open-open-fieldname-conflict_issue173">
<output-dir compare="Text">open-open-fieldname-conflict_issue173</output-dir>
- <expected-error>Open fields 0 and 1 have the same field name "name"</expected-error>
<source-location>false</source-location>
</compilation-unit>
</test-case>
@@ -216,4 +214,18 @@
<output-dir compare="Text">query-ASTERIXDB-2529</output-dir>
</compilation-unit>
</test-case>
-</test-group>
+ <test-case FilePath="objects" check-warnings="true">
+ <compilation-unit name="object_duplicate_fields">
+ <output-dir compare="Text">object_duplicate_fields</output-dir>
+ <expected-warn>Duplicate field name "name" (in line 25, at column 1)</expected-warn>
+ <expected-warn>Duplicate field name "Name" (in line 27, at column 1)</expected-warn>
+ <expected-warn>Duplicate field name "name" (in line 29, at column 1)</expected-warn>
+ <expected-warn>Duplicate field name "name" (in line 20, at column 30)</expected-warn>
+ <expected-warn>Duplicate field name "id" (in line 20, at column 56)</expected-warn>
+ <expected-warn>Duplicate field name "f1" (in line 20, at column 70)</expected-warn>
+ <expected-warn>Duplicate field name "id" (in line 20, at column 36)</expected-warn>
+ <expected-warn>Duplicate field name "f1" (in line 20, at column 83)</expected-warn>
+ <expected-warn>Duplicate field name "fname1" (in line 23, at column 45)</expected-warn>
+ </compilation-unit>
+ </test-case>
+</test-group>
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.1.ddl.sqlpp
index 01512fb..d73111a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.1.ddl.sqlpp
@@ -18,7 +18,7 @@
*/
/*
* Description : Tests whether a conflict between two closed field names are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
+ * Expected Result: Success (ignoring duplicates) with a warning reporting that there is a duplicate field name "name"
* Author: zheilbron
*/
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.sqlpp
index f205e1a..8ce4fab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.sqlpp
@@ -18,7 +18,7 @@
*/
/*
* Description : Tests whether a conflict between two closed field names are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
+ * Expected Result: Success (ignoring duplicates) with a warning reporting that there is a duplicate field name "name"
* Author: zheilbron
*/
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.3.query.sqlpp
index a7acd46..efc6d34 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.3.query.sqlpp
@@ -18,7 +18,7 @@
*/
/*
* Description : Tests whether a conflict between two closed field names are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
+ * Expected Result: Success (ignoring duplicates) with a warning reporting that there is a duplicate field name "name"
* Author: zheilbron
*/
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/no_fieldname_constr_negative/no_fieldname_constr_negative.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/no_fieldname_constr/no_fieldname_constr.3.query.sqlpp
similarity index 91%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/no_fieldname_constr_negative/no_fieldname_constr_negative.2.query.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/no_fieldname_constr/no_fieldname_constr.3.query.sqlpp
index c20cc84..e6ec490 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/no_fieldname_constr_negative/no_fieldname_constr_negative.2.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/no_fieldname_constr/no_fieldname_constr.3.query.sqlpp
@@ -19,11 +19,11 @@
/*
* Description : Testing object constructor without field names
- * Expected Res : Failure: Duplicate field name
+ * Expected Res : Success with a warning reporting that there is a duplicate field name
*/
from (
from range(1, 2) x
select value { "a": x, "b": { "a": x + 1 } }
) x
-select value { x.a, x.b.a }
+select value { x.a, x.b.a }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.01.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.01.ddl.sqlpp
new file mode 100644
index 0000000..3fd2ff7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.01.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description : Tests that duplicate fields are ignored and only one field is picked
+ * Expected Result: Success with a warning reporting that there is a duplicate field name
+ */
+
+DROP dataverse test if exists;
+CREATE dataverse test;
+
+use test;
+
+create type opentype as {
+ fname1 : string,
+ fname2 : string
+};
+
+create dataset testds(opentype) primary key fname1;
+
+create dataset ds(opentype) primary key fname1 with {
+"merge-policy": {"name":"no-merge"},
+"merge-policy": {
+ "name": "prefix",
+ "parameters": { "max-mergable-component-size": 32768, "max-tolerance-component-count": 32 }
+ }
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.02.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.02.update.sqlpp
new file mode 100644
index 0000000..c43d7b5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.02.update.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into testds
+select element {'fname1':'val1','fname2':'val2', 'fname1':'val3', 'fname2':'val4', 'fname1':'val5'};
+
+insert into testds
+select element {'fname1':'val99','fname2':'val2', 'fname1':'val3', 'fname3': 7, 'fname2':'val4', 'fname1':'val5'};
+
+insert into testds
+select element {'fname1':'val88','fname1':'val3', 'fname3': 7, 'fname2':'val4', lowercase('Fname3'):10, 'fname2': 12, lowercase('Fname1'):'xx', lowercase('Fname1'):10};
+
+insert into testds
+select element {'fname1':'val77', lowercase('Fname3'):10, 'fname1':'val3', 'fname2':'val4', 'fname3': 7, 'fname2': 12, lowercase('Fname1'):'xx', lowercase('Fname1'):10};
+
+insert into testds
+select element {'fname1':'val66', lowercase('Fname5'):22, 'fname1':'val3', 'fname2':'val4', 'fname3': 7, 'fname2': 12, lowercase('Fname1'):'xx', lowercase('Fname1'):10};
+
+insert into testds
+select element {'fname1':'val55', lowercase('Fname5'):22, 'fname1':'val3', 'fname2':'val4', 'fname3': 7, lowercase('Fname5'):11, 'fname2': 12, lowercase('Fname1'):'xx', lowercase('Fname1'):10};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.03.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.03.query.sqlpp
index 859b0b3..190867d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.03.query.sqlpp
@@ -16,10 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-/*
- * Description : Tests whether a conflict between two closed field names are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
- * Author: zheilbron
- */
-// no inserts, deletes
+use test;
+
+select value x from testds as x order by x.fname1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.04.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.04.query.sqlpp
index 859b0b3..39d6369 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.04.query.sqlpp
@@ -16,10 +16,15 @@
* specific language governing permissions and limitations
* under the License.
*/
-/*
- * Description : Tests whether a conflict between two closed field names are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
- * Author: zheilbron
- */
-// no inserts, deletes
+use test;
+
+select value {
+"name": "Sam",
+"Name": "John",
+"name": "Sam2",
+"id": 4,
+"Name": "John2",
+"name1": "Sam",
+"name": "Sam3"
+};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.05.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.05.query.sqlpp
index 859b0b3..4de754f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.05.query.sqlpp
@@ -16,10 +16,7 @@
* specific language governing permissions and limitations
* under the License.
*/
-/*
- * Description : Tests whether a conflict between two closed field names are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
- * Author: zheilbron
- */
-// no inserts, deletes
+use test;
+
+select d.CompactionPolicy, d.CompactionPolicyProperties from `Metadata`.`Dataset` d where d.DataverseName = "test" and d.DatasetName = "ds";
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.06.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.06.query.sqlpp
index 859b0b3..0df4cd6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.06.query.sqlpp
@@ -16,10 +16,5 @@
* specific language governing permissions and limitations
* under the License.
*/
-/*
- * Description : Tests whether a conflict between two closed field names are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
- * Author: zheilbron
- */
-// no inserts, deletes
+select value {"name": "Sam", lowercase('NAME'): "John"};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.07.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.07.query.sqlpp
index 859b0b3..b6e9e5b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.07.query.sqlpp
@@ -16,10 +16,5 @@
* specific language governing permissions and limitations
* under the License.
*/
-/*
- * Description : Tests whether a conflict between two closed field names are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
- * Author: zheilbron
- */
-// no inserts, deletes
+select value {"name": "Sam", "f1": {"id": 3, "id2": 7, "id": 8}};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.08.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.08.query.sqlpp
index 859b0b3..14bd59c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.08.query.sqlpp
@@ -16,10 +16,5 @@
* specific language governing permissions and limitations
* under the License.
*/
-/*
- * Description : Tests whether a conflict between two closed field names are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
- * Author: zheilbron
- */
-// no inserts, deletes
+select value {"name": "Sam", "f1": {"id": 3, "id2": 5}, "f2": "str", "f1": 8};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.09.query.sqlpp
similarity index 77%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.09.query.sqlpp
index 859b0b3..48db565 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.09.query.sqlpp
@@ -16,10 +16,5 @@
* specific language governing permissions and limitations
* under the License.
*/
-/*
- * Description : Tests whether a conflict between two closed field names are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
- * Author: zheilbron
- */
-// no inserts, deletes
+select value {"name": "Sam", "f1": {"id": 3, "id2": 5, "id": "sth"}, "f2": "str", "f1": 8};
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.10.query.sqlpp
similarity index 77%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.10.query.sqlpp
index 859b0b3..386b187 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.2.update.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_duplicate_fields/object_duplicate_fields.10.query.sqlpp
@@ -16,10 +16,9 @@
* specific language governing permissions and limitations
* under the License.
*/
-/*
- * Description : Tests whether a conflict between two closed field names are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
- * Author: zheilbron
- */
-// no inserts, deletes
+use test;
+
+from testds as x
+where x = {'fname1':'val1','fname2':'val2', 'fname1': 4}
+select value x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/open-closed-fieldname-conflict_issue173/open-closed-fieldname-conflict_issue173.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/open-closed-fieldname-conflict_issue173/open-closed-fieldname-conflict_issue173.3.query.sqlpp
index f15d2d1..6737f52 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/open-closed-fieldname-conflict_issue173/open-closed-fieldname-conflict_issue173.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/open-closed-fieldname-conflict_issue173/open-closed-fieldname-conflict_issue173.3.query.sqlpp
@@ -18,13 +18,11 @@
*/
/*
* Description : Tests whether a conflict between an open and closed field name are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
+ * Expected Result: Success (ignoring duplicates) with a warning reporting that there is a duplicate field name "name"
* Author: zheilbron
*/
use test;
-
select element {x.fname:'smith',test.lowercase('NAME'):'john'}
-from testds as x
-;
+from testds as x;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/open-open-fieldname-conflict_issue173/open-open-fieldname-conflict_issue173.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/open-open-fieldname-conflict_issue173/open-open-fieldname-conflict_issue173.3.query.sqlpp
index b83b673..b944566 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/open-open-fieldname-conflict_issue173/open-open-fieldname-conflict_issue173.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/open-open-fieldname-conflict_issue173/open-open-fieldname-conflict_issue173.3.query.sqlpp
@@ -18,7 +18,7 @@
*/
/*
* Description : Tests whether a conflict between two open field names are detected
- * Expected Result: An error reporting that there is a duplicate field name "name"
+ * Expected Result: Success (ignoring duplicates) with a warning reporting that there is a duplicate field name "name"
* Author: zheilbron
*/
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.2.adm
new file mode 100644
index 0000000..49b483b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.2.adm
@@ -0,0 +1 @@
+{ "b": 2, "c": [ 20 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.3.adm
new file mode 100644
index 0000000..75c8b4d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/query-ASTERIXDB-2446/query-ASTERIXDB-2446.3.adm
@@ -0,0 +1 @@
+{ "e": [ 20 ], "d1": 10 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.3.adm
new file mode 100644
index 0000000..0052fbf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.3.adm
@@ -0,0 +1 @@
+{ "name": "john" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/no_fieldname_constr/no_fieldname_constr.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/no_fieldname_constr/no_fieldname_constr.3.adm
new file mode 100644
index 0000000..27c6c51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/no_fieldname_constr/no_fieldname_constr.3.adm
@@ -0,0 +1,2 @@
+{ "a": 1 }
+{ "a": 2 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.03.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.03.adm
new file mode 100644
index 0000000..03ed756
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.03.adm
@@ -0,0 +1,6 @@
+{ "fname1": "val1", "fname2": "val2" }
+{ "fname1": "val55", "fname2": "val4", "fname5": 22, "fname3": 7 }
+{ "fname1": "val66", "fname2": "val4", "fname5": 22, "fname3": 7 }
+{ "fname1": "val77", "fname2": "val4", "fname3": 7 }
+{ "fname1": "val88", "fname2": "val4", "fname3": 7 }
+{ "fname1": "val99", "fname2": "val2", "fname3": 7 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.04.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.04.adm
new file mode 100644
index 0000000..21de15e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.04.adm
@@ -0,0 +1 @@
+{ "name": "Sam", "Name": "John", "id": 4, "name1": "Sam" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.05.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.05.adm
new file mode 100644
index 0000000..cff5035
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.05.adm
@@ -0,0 +1 @@
+{ "CompactionPolicy": "no-merge", "CompactionPolicyProperties": [ ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.06.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.06.adm
new file mode 100644
index 0000000..5efebe5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.06.adm
@@ -0,0 +1 @@
+{ "name": "Sam" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.07.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.07.adm
new file mode 100644
index 0000000..bc1e8bb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.07.adm
@@ -0,0 +1 @@
+{ "name": "Sam", "f1": { "id": 3, "id2": 7 } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.08.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.08.adm
new file mode 100644
index 0000000..f5e14fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.08.adm
@@ -0,0 +1 @@
+{ "name": "Sam", "f1": { "id": 3, "id2": 5 }, "f2": "str" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.09.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.09.adm
new file mode 100644
index 0000000..f5e14fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.09.adm
@@ -0,0 +1 @@
+{ "name": "Sam", "f1": { "id": 3, "id2": 5 }, "f2": "str" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.10.adm
new file mode 100644
index 0000000..830a761
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_duplicate_fields/object_duplicate_fields.10.adm
@@ -0,0 +1 @@
+{ "fname1": "val1", "fname2": "val2" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/open-closed-fieldname-conflict_issue173/open-closed-fieldname-conflict_issue173.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/open-closed-fieldname-conflict_issue173/open-closed-fieldname-conflict_issue173.3.adm
new file mode 100644
index 0000000..0052fbf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/open-closed-fieldname-conflict_issue173/open-closed-fieldname-conflict_issue173.3.adm
@@ -0,0 +1 @@
+{ "name": "john" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/open-open-fieldname-conflict_issue173/open-open-fieldname-conflict_issue173.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/open-open-fieldname-conflict_issue173/open-open-fieldname-conflict_issue173.3.adm
new file mode 100644
index 0000000..0052fbf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/open-open-fieldname-conflict_issue173/open-open-fieldname-conflict_issue173.3.adm
@@ -0,0 +1 @@
+{ "name": "john" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.3.ast
index a34f71f..93be181 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/objects/closed-closed-fieldname-conflict_issue173/closed-closed-fieldname-conflict_issue173.3.ast
@@ -6,9 +6,4 @@
:
LiteralExpr [STRING] [john]
)
- (
- LiteralExpr [STRING] [name]
- :
- LiteralExpr [STRING] [smith]
- )
]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index ccd8f5a..43c8445 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -23,7 +23,9 @@
<!ENTITY TemporalQueries SYSTEM "queries_sqlpp/temporal/TemporalQueries.xml">
]>
<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries_sqlpp" QueryFileExtension=".sqlpp" SourceLocation="true">
+ &ObjectsQueries;
&AsyncDeferredQueries;
+ &GeoQueries;
&TemporalQueries;
<test-group name="flwor">
<test-case FilePath="flwor">
@@ -99,12 +101,12 @@
<expected-error>Duplicate alias definitions: s1</expected-error>
</compilation-unit>
</test-case>
- <test-case FilePath="flwor">
+ <test-case FilePath="flwor" check-warnings="true">
<compilation-unit name="query-ASTERIXDB-2446">
- <output-dir compare="Text">query-ASTERIXDB-883</output-dir>
+ <output-dir compare="Text">query-ASTERIXDB-2446</output-dir>
<expected-error>ASX0013: Duplicate field name "a"</expected-error>
- <expected-error>ASX0013: Duplicate field name "c"</expected-error>
- <expected-error>ASX0013: Duplicate field name "e"</expected-error>
+ <expected-warn>Duplicate field name "c" (in line 26, at column 84)</expected-warn>
+ <expected-warn>Duplicate field name "e" (in line 26, at column 116)</expected-warn>
</compilation-unit>
</test-case>
<test-case FilePath="flwor">
@@ -124,6 +126,11 @@
<output-dir compare="Text">select-let</output-dir>
</compilation-unit>
</test-case>
+ <test-case FilePath="flwor">
+ <compilation-unit name="let33">
+ <output-dir compare="Text">let33</output-dir>
+ </compilation-unit>
+ </test-case>
</test-group>
<test-group name="sorting">
<test-case FilePath="sorting">
@@ -188,11 +195,6 @@
</compilation-unit>
</test-case>
</test-group>
- <test-case FilePath="flwor">
- <compilation-unit name="let33">
- <output-dir compare="Text">let33</output-dir>
- </compilation-unit>
- </test-case>
<test-group name="aggregate">
<test-case FilePath="aggregate">
<compilation-unit name="issue531_string_min_max">
@@ -7899,7 +7901,6 @@
</compilation-unit>
</test-case> -->
</test-group>
- &ObjectsQueries;
<test-group name="resolution">
<test-case FilePath="resolution">
<compilation-unit name="conflict-field-dataset">
@@ -12357,7 +12358,6 @@
</compilation-unit>
</test-case>
</test-group>
- &GeoQueries;
<test-group name="compression">
<test-case FilePath="compression">
<compilation-unit name="incompressible-pages/large-page">
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
index c358444..6a34f65 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/LangRecordParseUtil.java
@@ -18,9 +18,7 @@
*/
package org.apache.asterix.lang.common.util;
-import java.io.Serializable;
import java.util.Collections;
-import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
@@ -44,7 +42,6 @@
import org.apache.asterix.om.base.ABoolean;
import org.apache.asterix.om.base.ADouble;
import org.apache.asterix.om.base.AInt64;
-import org.apache.asterix.om.base.AMutableString;
import org.apache.asterix.om.base.AOrderedList;
import org.apache.asterix.om.base.ARecord;
import org.apache.asterix.om.base.AString;
@@ -55,6 +52,7 @@
import org.apache.hyracks.algebricks.common.utils.Pair;
import org.apache.hyracks.api.exceptions.HyracksDataException;
+// TODO(ali): all the functionality here is the same as the ones in ExpressionUtils
public class LangRecordParseUtil {
private static final String NOT_ALLOWED_EXPRESSIONS_ERROR_MESSAGE =
"JSON record can only have expressions [%1$s, %2$s, %3$s]";
@@ -72,35 +70,26 @@
return parseList((ListConstructor) expr);
default:
throw new HyracksDataException(ErrorCode.ASTERIX, ErrorCode.PARSE_ERROR,
- NOT_ALLOWED_EXPRESSIONS_ERROR_MESSAGE,
- new Serializable[] { Expression.Kind.LITERAL_EXPRESSION.toString(),
- Expression.Kind.RECORD_CONSTRUCTOR_EXPRESSION.toString(),
- Expression.Kind.LIST_CONSTRUCTOR_EXPRESSION.toString() });
+ NOT_ALLOWED_EXPRESSIONS_ERROR_MESSAGE, Expression.Kind.LITERAL_EXPRESSION.toString(),
+ Expression.Kind.RECORD_CONSTRUCTOR_EXPRESSION.toString(),
+ Expression.Kind.LIST_CONSTRUCTOR_EXPRESSION.toString());
}
}
public static AdmObjectNode parseRecord(RecordConstructor recordValue, List<Pair<String, String>> defaults)
throws HyracksDataException, CompilationException {
AdmObjectNode record = new AdmObjectNode();
- AMutableString fieldNameString = new AMutableString(null);
List<FieldBinding> fbList = recordValue.getFbList();
- HashSet<String> fieldNames = new HashSet<>();
for (FieldBinding fb : fbList) {
// get key
- fieldNameString.setValue(exprToStringLiteral(fb.getLeftExpr()).getStringValue());
- if (!fieldNames.add(fieldNameString.getStringValue())) {
- throw new HyracksDataException(
- "Field " + fieldNameString.getStringValue() + " was specified multiple times");
- }
+ String key = exprToStringLiteral(fb.getLeftExpr()).getStringValue();
// get value
IAdmNode value = parseExpression(fb.getRightExpr());
- record.set(fieldNameString.getStringValue(), value);
+ record.set(key, value);
}
// defaults
for (Pair<String, String> kv : defaults) {
- if (!fieldNames.contains(kv.first)) {
- record.set(kv.first, new AdmStringNode(kv.second));
- }
+ record.set(kv.first, new AdmStringNode(kv.second));
}
return record;
}
@@ -180,7 +169,7 @@
}
}
- public static String aObjToString(IAObject aObj) throws AlgebricksException {
+ private static String aObjToString(IAObject aObj) throws AlgebricksException {
switch (aObj.getType().getTypeTag()) {
case DOUBLE:
return Double.toString(((ADouble) aObj).getDoubleValue());
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index 0909fe0..256e96d 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -2756,7 +2756,9 @@
throw new SqlppParseException(getSourceLocation(token), "Cannot infer field name");
}
String generatedName = SqlppVariableUtil.toUserDefinedName(generatedIdentifier);
- return new FieldBinding(new LiteralExpr(new StringLiteral(generatedName)), left);
+ LiteralExpr generatedNameExpr = new LiteralExpr(new StringLiteral(generatedName));
+ generatedNameExpr.setSourceLocation(left.getSourceLocation());
+ return new FieldBinding(generatedNameExpr, left);
} else {
return new FieldBinding(left, right);
}
diff --git a/asterixdb/asterix-om/pom.xml b/asterixdb/asterix-om/pom.xml
index f536be1..fc1707f 100644
--- a/asterixdb/asterix-om/pom.xml
+++ b/asterixdb/asterix-om/pom.xml
@@ -134,5 +134,9 @@
<artifactId>log4j-api</artifactId>
<scope>test</scope>
</dependency>
+ <dependency>
+ <groupId>it.unimi.dsi</groupId>
+ <artifactId>fastutil</artifactId>
+ </dependency>
</dependencies>
</project>
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
index 742cfb3..58e59d6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/builders/RecordBuilder.java
@@ -19,8 +19,6 @@
package org.apache.asterix.builders;
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
import java.io.DataOutput;
import java.io.IOException;
import java.util.Arrays;
@@ -40,6 +38,8 @@
import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer;
+import it.unimi.dsi.fastutil.ints.IntOpenHashSet;
+
public class RecordBuilder implements IARecordBuilder {
private final static int DEFAULT_NUM_OPEN_FIELDS = 10;
private final UTF8StringSerializerDeserializer utf8SerDer = new UTF8StringSerializerDeserializer();
@@ -64,6 +64,7 @@
private int[] openFieldNameLengths;
private int numberOfOpenFields;
private final RuntimeRecordTypeInfo recTypeInfo;
+ private final IntOpenHashSet fieldNamesHashes;
public RecordBuilder() {
this.closedPartOutputStream = new ByteArrayAccessibleOutputStream();
@@ -79,6 +80,7 @@
this.openPartOffsetArraySize = 0;
this.offsetPosition = 0;
this.recTypeInfo = new RuntimeRecordTypeInfo();
+ this.fieldNamesHashes = new IntOpenHashSet();
}
@Override
@@ -88,6 +90,7 @@
this.openPartOutputStream.reset();
this.numberOfOpenFields = 0;
this.offsetPosition = 0;
+ this.fieldNamesHashes.clear();
if (nullBitMap != null) {
// A default null byte is 10101010 (0xAA):
// the null bit is 1, which means the field is not a null,
@@ -105,6 +108,7 @@
this.numberOfClosedFields = 0;
this.numberOfOpenFields = 0;
this.offsetPosition = 0;
+ this.fieldNamesHashes.clear();
if (recType != null) {
this.isOpen = recType.isOpen();
this.containsOptionalField = NonTaggedFormatUtil.hasOptionalField(recType);
@@ -182,27 +186,34 @@
if (data[offset] == ATypeTag.SERIALIZED_MISSING_TYPE_TAG) {
return;
}
+ // ignore adding duplicate fields
+ byte[] nameBytes = name.getByteArray();
+ int nameStart = name.getStartOffset() + 1;
+ int nameLength = name.getLength() - 1;
+ if (recType != null && recTypeInfo.getFieldIndex(nameBytes, nameStart, nameLength) >= 0) {
+ // TODO(ali): issue a warning
+ return;
+ }
+ int fieldNameHashCode = utf8HashFunction.hash(nameBytes, nameStart, nameLength);
+ if (!fieldNamesHashes.add(fieldNameHashCode)) {
+ for (int i = 0; i < numberOfOpenFields; i++) {
+ if (isDuplicate(nameBytes, nameStart, nameLength, fieldNameHashCode, i)) {
+ // TODO(ali): issue a warning
+ return;
+ }
+ }
+ }
if (numberOfOpenFields == openPartOffsets.length) {
openPartOffsets = Arrays.copyOf(openPartOffsets, openPartOffsets.length + DEFAULT_NUM_OPEN_FIELDS);
openFieldNameLengths =
Arrays.copyOf(openFieldNameLengths, openFieldNameLengths.length + DEFAULT_NUM_OPEN_FIELDS);
}
- int fieldNameHashCode =
- utf8HashFunction.hash(name.getByteArray(), name.getStartOffset() + 1, name.getLength() - 1);
- if (recType != null) {
- int cFieldPos;
- cFieldPos = recTypeInfo.getFieldIndex(name.getByteArray(), name.getStartOffset() + 1, name.getLength() - 1);
- if (cFieldPos >= 0) {
- throw new HyracksDataException("Open field \"" + recType.getFieldNames()[cFieldPos]
- + "\" has the same field name as closed field at index " + cFieldPos);
- }
- }
openPartOffsets[this.numberOfOpenFields] = fieldNameHashCode;
openPartOffsets[this.numberOfOpenFields] = openPartOffsets[numberOfOpenFields] << 32;
openPartOffsets[numberOfOpenFields] += openPartOutputStream.size();
- openFieldNameLengths[numberOfOpenFields++] = name.getLength() - 1;
- openPartOutputStream.write(name.getByteArray(), name.getStartOffset() + 1, name.getLength() - 1);
- openPartOutputStream.write(value.getByteArray(), value.getStartOffset(), value.getLength());
+ openFieldNameLengths[numberOfOpenFields++] = nameLength;
+ openPartOutputStream.write(nameBytes, nameStart, nameLength);
+ openPartOutputStream.write(data, offset, value.getLength());
}
@Override
@@ -220,27 +231,6 @@
// field names with the same hash code should be adjacent to each other after sorting
Arrays.sort(this.openPartOffsets, 0, numberOfOpenFields);
- if (numberOfOpenFields > 1) {
- byte[] openBytes = openPartOutputStream.getByteArray();
- for (int i = 0, k = 1; i < numberOfOpenFields - 1;) {
- if (utf8Comparator.compare(openBytes, (int) openPartOffsets[i], openFieldNameLengths[i], openBytes,
- (int) openPartOffsets[k], openFieldNameLengths[k]) == 0) {
- String field = utf8SerDer.deserialize(new DataInputStream(new ByteArrayInputStream(openBytes,
- (int) openPartOffsets[i], openFieldNameLengths[i])));
- throw new HyracksDataException(
- "Open fields " + i + " and " + k + " have the same field name \"" + field + "\"");
- }
- if (sameHashes(openPartOffsets, i, k) && k < numberOfOpenFields - 1) {
- // keep comparing the current field i with the next field k
- k++;
- } else {
- // the current field i has no duplicates; move to the adjacent one
- i++;
- k = i + 1;
- }
- }
- }
-
openPartOffset = h + numberOfSchemaFields * 4 + closedPartOutputStream.size();
int fieldNameHashCode;
for (int i = 0; i < numberOfOpenFields; i++) {
@@ -258,10 +248,6 @@
writeRecord(out, writeTypeTag, h, recordLength);
}
- private static boolean sameHashes(long[] hashAndOffset, int fieldId1, int fieldId2) {
- return (int) (hashAndOffset[fieldId1] >>> 32) == (int) (hashAndOffset[fieldId2] >>> 32);
- }
-
private void writeRecord(DataOutput out, boolean writeTypeTag, int headerSize, int recordLength)
throws HyracksDataException {
try {
@@ -314,4 +300,11 @@
public IBinaryComparator getFieldNameComparator() {
return utf8Comparator;
}
+
+ private boolean isDuplicate(byte[] fName, int fStart, int fLen, int fNameHash, int otherFieldIdx)
+ throws HyracksDataException {
+ return ((int) (openPartOffsets[otherFieldIdx] >>> 32) == fNameHash)
+ && (utf8Comparator.compare(openPartOutputStream.getByteArray(), (int) openPartOffsets[otherFieldIdx],
+ openFieldNameLengths[otherFieldIdx], fName, fStart, fLen) == 0);
+ }
}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java
index 6d2658b..26024b4 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/object/base/AdmObjectNode.java
@@ -27,7 +27,6 @@
import java.util.Set;
import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.common.exceptions.CompilationException;
import org.apache.asterix.common.exceptions.ErrorCode;
import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
import org.apache.asterix.om.base.AString;
@@ -76,12 +75,13 @@
return children.entrySet();
}
- public AdmObjectNode set(String fieldName, IAdmNode value) throws CompilationException {
+ public AdmObjectNode set(String fieldName, IAdmNode value) {
if (value == null) {
value = AdmNullNode.INSTANCE; // NOSONAR
}
if (children.containsKey(fieldName)) {
- throw new CompilationException(ErrorCode.DUPLICATE_FIELD_NAME, fieldName);
+ // TODO(ali): find a way to issue a warning
+ return this;
}
children.put(fieldName, value);
return this;